From 0f9271a34979fe44a31e4a950a0823b4f3ff3338 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 12:10:06 +0000 Subject: [PATCH 01/59] Passing sort description apart of fill description --- src/Interpreters/FillingRow.cpp | 10 ++++--- src/Interpreters/FillingRow.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++----- src/Planner/Planner.cpp | 6 +++-- src/Processors/QueryPlan/FillingStep.cpp | 14 +++++++--- src/Processors/QueryPlan/FillingStep.h | 7 ++++- .../Transforms/FillingTransform.cpp | 26 +++++++++++++++---- src/Processors/Transforms/FillingTransform.h | 13 +++++++--- 8 files changed, 65 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 05795842902..98197533aa1 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -67,12 +67,14 @@ bool FillingRow::next(const FillingRow & to_row) /// we need to generate rows up to 'fill_to' value. for (size_t i = row_size - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + auto & fill_column_desc = getFillDescription(i); + + if (fill_column_desc.fill_to.isNull() || row[i].isNull()) continue; - auto next_value = row[i]; - getFillDescription(i).step_func(next_value); - if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) + Field next_value = row[i]; + fill_column_desc.step_func(next_value); + if (less(next_value, fill_column_desc.fill_to, getDirection(i))) { row[i] = next_value; initFromDefaults(i + 1); diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index c56bd875151..9f414d92dd3 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -32,6 +32,7 @@ public: int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } + const FillColumnDescription & getFillDescription(size_t index) const { return sort_description[index].fill_description; } String dump() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index afd34cb044e..e36a1287503 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2995,20 +2995,21 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); - SortDescription fill_descr; - for (auto & desc : order_descr) + SortDescription sort_description = getSortDescription(query, context); + SortDescription fill_description; + for (auto & desc : sort_description) { if (desc.with_fill) - fill_descr.push_back(desc); + fill_description.push_back(desc); } - if (fill_descr.empty()) + if (fill_description.empty()) return; InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); + auto filling_step = std::make_unique( + query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_descr); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d036c895fbb..e3a4d01a46a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -622,7 +622,8 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); + auto filling_step = std::make_unique( + query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } @@ -1432,7 +1433,8 @@ void Planner::buildPlanForQueryNode() addLimitByStep(query_plan, limit_by_analysis_result, query_node); } - addWithFillStepIfNeeded(query_plan, query_analysis_result, planner_context, query_node); + if (query_node.hasOrderBy()) + addWithFillStepIfNeeded(query_plan, query_analysis_result, planner_context, query_node); bool apply_offset = query_processing_info.getToStage() != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 20d7d6d0f8f..37c5307fcce 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -27,9 +27,15 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_) +FillingStep::FillingStep( + const DataStream & input_stream_, + SortDescription sort_description_, + SortDescription fill_description_, + InterpolateDescriptionPtr interpolate_description_) : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) - , sort_description(std::move(sort_description_)), interpolate_description(interpolate_description_) + , sort_description(std::move(sort_description_)) + , fill_description(std::move(fill_description_)) + , interpolate_description(interpolate_description_) { if (!input_stream_.has_single_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); @@ -40,9 +46,9 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return std::make_shared(header, sort_description); + return std::make_shared(header, fill_description); - return std::make_shared(header, sort_description, std::move(interpolate_description)); + return std::make_shared(header, sort_description, fill_description, std::move(interpolate_description)); }); } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 4e1b5b3654d..0febbbb8eb0 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -10,7 +10,11 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_); + FillingStep( + const DataStream & input_stream_, + SortDescription sort_description_, + SortDescription fill_description_, + InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "Filling"; } @@ -25,6 +29,7 @@ private: void updateOutputStream() override; SortDescription sort_description; + SortDescription fill_description; InterpolateDescriptionPtr interpolate_description; }; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4c5b74aad7c..4fc2b953a4f 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -187,9 +187,10 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) + const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) , sort_description(sort_description_) + , fill_description(fill_description_) , interpolate_description(interpolate_description_) , filling_row(sort_description_) , next_row(sort_description_) @@ -198,14 +199,14 @@ FillingTransform::FillingTransform( interpolate_actions = std::make_shared(interpolate_description->actions); std::vector is_fill_column(header_.columns()); - for (size_t i = 0, size = sort_description.size(); i < size; ++i) + for (size_t i = 0, size = fill_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->result_columns_set.contains(sort_description[i].column_name)) + if (interpolate_description && interpolate_description->result_columns_set.contains(fill_description[i].column_name)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", - sort_description[i].column_name); + fill_description[i].column_name); - size_t block_position = header_.getPositionByName(sort_description[i].column_name); + size_t block_position = header_.getPositionByName(fill_description[i].column_name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); @@ -232,6 +233,21 @@ FillingTransform::FillingTransform( if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); + /// build sorting prefix for first fill column + for (const auto & desc : sort_description) + { + if (desc.column_name == fill_description[0].column_name) + break; + + size_t pos = header_.getPositionByName(desc.column_name); + sort_prefix_positions.push_back(pos); + + sort_prefix.push_back(desc); + } + logDebug("sort prefix", dumpSortDescription(sort_prefix)); + + /// TODO: check conflict in positions between interpolate and sorting prefix columns + size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) { diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 7aa5e4c1e8a..021784c263d 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -16,7 +16,11 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_); + FillingTransform( + const Block & header_, + const SortDescription & sort_description_, + const SortDescription & fill_description_, + InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "FillingTransform"; } @@ -25,7 +29,7 @@ public: static Block transformHeader(Block header, const SortDescription & sort_description); protected: - void transform(Chunk & Chunk) override; + void transform(Chunk & chunk) override; private: void saveLastRow(const MutableColumns & result_columns); @@ -46,7 +50,9 @@ private: const Columns & input_columns, MutableColumns & result_columns); - const SortDescription sort_description; /// Contains only columns with WITH FILL. + const SortDescription sort_description; + const SortDescription fill_description; /// Contains only columns with WITH FILL. + SortDescription sort_prefix; const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns FillingRow filling_row; /// Current row, which is used to fill gaps. @@ -56,6 +62,7 @@ private: Positions fill_column_positions; Positions interpolate_column_positions; Positions other_column_positions; + Positions sort_prefix_positions; std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; From e7c3d74b7ce6a9703a0b71b0ec9cdffa4739fc9a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 21:37:15 +0000 Subject: [PATCH 02/59] Fix filling_row & next_row construction --- .../Transforms/FillingTransform.cpp | 58 +++++++++++++++++-- src/Processors/Transforms/FillingTransform.h | 16 ++++- 2 files changed, 69 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4fc2b953a4f..f3e32c3cd89 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -14,7 +14,7 @@ namespace DB { -constexpr bool debug_logging_enabled = false; +constexpr bool debug_logging_enabled = true; template void logDebug(String key, const T & value, const char * separator = " : ") @@ -192,8 +192,8 @@ FillingTransform::FillingTransform( , sort_description(sort_description_) , fill_description(fill_description_) , interpolate_description(interpolate_description_) - , filling_row(sort_description_) - , next_row(sort_description_) + , filling_row(fill_description_) + , next_row(fill_description_) { if (interpolate_description) interpolate_actions = std::make_shared(interpolate_description->actions); @@ -227,6 +227,7 @@ FillingTransform::FillingTransform( "WITH FILL bound values cannot be negative for unsigned type {}", type->getName()); } } + logDebug("fill description", dumpSortDescription(fill_description)); std::set unique_positions; for (auto pos : fill_column_positions) @@ -369,7 +370,7 @@ static void initColumnsByPositions( MutableColumnRawPtrs & output_columns_by_position, const std::vector & positions) { - for (size_t pos : positions) + for (const size_t pos : positions) { input_columns_by_positions.push_back(input_columns[pos]); output_columns_by_position.push_back(output_columns[pos].get()); @@ -380,10 +381,12 @@ void FillingTransform::initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, + Columns & ,//input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, + MutableColumnRawPtrs & , //output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns) { Columns non_const_columns; @@ -398,6 +401,7 @@ void FillingTransform::initColumns( initColumnsByPositions(non_const_columns, input_fill_columns, output_columns, output_fill_columns, fill_column_positions); initColumnsByPositions( non_const_columns, input_interpolate_columns, output_columns, output_interpolate_columns, interpolate_column_positions); + // initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); initColumnsByPositions(non_const_columns, input_other_columns, output_columns, output_other_columns, other_column_positions); } @@ -423,19 +427,23 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut Columns input_fill_columns; Columns input_interpolate_columns; + Columns input_sort_prefix_columns; Columns input_other_columns; MutableColumnRawPtrs res_fill_columns; MutableColumnRawPtrs res_interpolate_columns; + MutableColumnRawPtrs res_sort_prefix_columns; MutableColumnRawPtrs res_other_columns; initColumns( input_columns, input_fill_columns, input_interpolate_columns, + input_sort_prefix_columns, input_other_columns, result_columns, res_fill_columns, res_interpolate_columns, + res_sort_prefix_columns, res_other_columns); if (first) @@ -457,6 +465,43 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut return true; } +// void FillingTransform::transformRange( +// const Columns & input_fill_columns, +// const Columns & input_interpolate_columns, +// const Columns & input_sort_prefix_columns, +// const Columns & input_other_columns, +// const MutableColumns & result_columns, +// const MutableColumnRawPtrs & res_fill_columns, +// const MutableColumnRawPtrs & res_interpolate_columns, +// const MutableColumnRawPtrs & res_sort_prefix_columns, +// const MutableColumnRawPtrs & res_other_columns, +// std::pair range) +// { +// Block interpolate_block; +// if (first) +// { +// for (size_t i = 0, size = filling_row.size(); i < size; ++i) +// { +// auto current_value = (*input_fill_columns[i])[0]; +// const auto & fill_from = filling_row.getFillDescription(i).fill_from; + +// if (!fill_from.isNull() && !equals(current_value, fill_from)) +// { +// filling_row.initFromDefaults(i); +// if (less(fill_from, current_value, filling_row.getDirection(i))) +// { +// interpolate(result_columns, interpolate_block); +// insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); +// } +// break; +// } +// filling_row[i] = current_value; +// } +// first = false; +// } + +// } + void FillingTransform::transform(Chunk & chunk) { logDebug("new chunk rows", chunk.getNumRows()); @@ -469,9 +514,11 @@ void FillingTransform::transform(Chunk & chunk) Columns input_fill_columns; Columns input_interpolate_columns; + Columns input_sort_prefix_columns; Columns input_other_columns; MutableColumnRawPtrs res_fill_columns; MutableColumnRawPtrs res_interpolate_columns; + MutableColumnRawPtrs res_sort_prefix_columns; MutableColumnRawPtrs res_other_columns; MutableColumns result_columns; @@ -501,10 +548,12 @@ void FillingTransform::transform(Chunk & chunk) input_columns, input_fill_columns, input_interpolate_columns, + input_sort_prefix_columns, input_other_columns, result_columns, res_fill_columns, res_interpolate_columns, + res_sort_prefix_columns, res_other_columns); if (first) @@ -566,6 +615,7 @@ void FillingTransform::transform(Chunk & chunk) copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); + // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 021784c263d..dac92e43231 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -32,18 +32,32 @@ protected: void transform(Chunk & chunk) override; private: + using MutableColumnRawPtrs = std::vector; + void transformRange( + const Columns & input_fill_columns, + const Columns & input_interpolate_columns, + const Columns & input_sort_prefix_columns, + const Columns & input_other_columns, + const MutableColumns & result_columns, + const MutableColumnRawPtrs & res_fill_columns, + const MutableColumnRawPtrs & res_interpolate_columns, + const MutableColumnRawPtrs & res_sort_prefix_columns, + const MutableColumnRawPtrs & res_other_columns, + std::pair range); + void saveLastRow(const MutableColumns & result_columns); void interpolate(const MutableColumns & result_columns, Block & interpolate_block); - using MutableColumnRawPtrs = std::vector; void initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, + Columns & input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, + MutableColumnRawPtrs & output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns); bool generateSuffixIfNeeded( From 0370b6fbe68ec7c0965434d8934c2e00ba4e2d3e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 21:58:02 +0000 Subject: [PATCH 03/59] Add transformRange() --- .../Transforms/FillingTransform.cpp | 184 +++++++++--------- 1 file changed, 88 insertions(+), 96 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index f3e32c3cd89..99f9d55d130 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -465,42 +465,85 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut return true; } -// void FillingTransform::transformRange( -// const Columns & input_fill_columns, -// const Columns & input_interpolate_columns, -// const Columns & input_sort_prefix_columns, -// const Columns & input_other_columns, -// const MutableColumns & result_columns, -// const MutableColumnRawPtrs & res_fill_columns, -// const MutableColumnRawPtrs & res_interpolate_columns, -// const MutableColumnRawPtrs & res_sort_prefix_columns, -// const MutableColumnRawPtrs & res_other_columns, -// std::pair range) -// { -// Block interpolate_block; -// if (first) -// { -// for (size_t i = 0, size = filling_row.size(); i < size; ++i) -// { -// auto current_value = (*input_fill_columns[i])[0]; -// const auto & fill_from = filling_row.getFillDescription(i).fill_from; +void FillingTransform::transformRange( + const Columns & input_fill_columns, + const Columns & input_interpolate_columns, + const Columns &, //input_sort_prefix_columns, + const Columns & input_other_columns, + const MutableColumns & result_columns, + const MutableColumnRawPtrs & res_fill_columns, + const MutableColumnRawPtrs & res_interpolate_columns, + const MutableColumnRawPtrs &, //res_sort_prefix_columns, + const MutableColumnRawPtrs & res_other_columns, + std::pair range) +{ + const size_t range_begin = range.first; + const size_t range_end = range.second; -// if (!fill_from.isNull() && !equals(current_value, fill_from)) -// { -// filling_row.initFromDefaults(i); -// if (less(fill_from, current_value, filling_row.getDirection(i))) -// { -// interpolate(result_columns, interpolate_block); -// insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); -// } -// break; -// } -// filling_row[i] = current_value; -// } -// first = false; -// } + Block interpolate_block; + if (first) + { + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + { + auto current_value = (*input_fill_columns[i])[0]; + const auto & fill_from = filling_row.getFillDescription(i).fill_from; -// } + if (!fill_from.isNull() && !equals(current_value, fill_from)) + { + filling_row.initFromDefaults(i); + if (less(fill_from, current_value, filling_row.getDirection(i))) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + break; + } + filling_row[i] = current_value; + } + first = false; + } + + for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) + { + logDebug("row", row_ind); + logDebug("filling_row", filling_row); + logDebug("next_row", next_row); + + bool should_insert_first = next_row < filling_row; + logDebug("should_insert_first", should_insert_first); + + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + { + auto current_value = (*input_fill_columns[i])[row_ind]; + const auto & fill_to = filling_row.getFillDescription(i).fill_to; + + if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) + next_row[i] = current_value; + else + next_row[i] = fill_to; + } + logDebug("next_row updated", next_row); + + /// A case, when at previous step row was initialized from defaults 'fill_from' values + /// and probably we need to insert it to block. + if (should_insert_first && filling_row < next_row) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + + while (filling_row.next(next_row)) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + + copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); + copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); + // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + copyRowFromColumns(res_other_columns, input_other_columns, row_ind); + } +} void FillingTransform::transform(Chunk & chunk) { @@ -556,68 +599,17 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - if (first) - { - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - { - auto current_value = (*input_fill_columns[i])[0]; - const auto & fill_from = filling_row.getFillDescription(i).fill_from; - - if (!fill_from.isNull() && !equals(current_value, fill_from)) - { - filling_row.initFromDefaults(i); - if (less(fill_from, current_value, filling_row.getDirection(i))) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - break; - } - filling_row[i] = current_value; - } - first = false; - } - - for (size_t row_ind = 0; row_ind < num_rows; ++row_ind) - { - logDebug("row", row_ind); - logDebug("filling_row", filling_row); - logDebug("next_row", next_row); - - bool should_insert_first = next_row < filling_row; - logDebug("should_insert_first", should_insert_first); - - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - { - auto current_value = (*input_fill_columns[i])[row_ind]; - const auto & fill_to = filling_row.getFillDescription(i).fill_to; - - if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) - next_row[i] = current_value; - else - next_row[i] = fill_to; - } - logDebug("next_row updated", next_row); - - /// A case, when at previous step row was initialized from defaults 'fill_from' values - /// and probably we need to insert it to block. - if (should_insert_first && filling_row < next_row) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - - while (filling_row.next(next_row)) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - - copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); - copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); - // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - copyRowFromColumns(res_other_columns, input_other_columns, row_ind); - } + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {0, num_rows}); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); From ea2970d344a9fee2e9f3b3d8d830a59f3c890673 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 5 May 2023 18:17:18 +0000 Subject: [PATCH 04/59] Fix: correct header for FillingTransform fixes 01921_with_fill_with_totals --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 99f9d55d130..93dca03e4ac 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -188,7 +188,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) + : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) , sort_description(sort_description_) , fill_description(fill_description_) , interpolate_description(interpolate_description_) From 06b0503fe8e8f37322056941aad301b119f58c2b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 5 May 2023 21:16:14 +0000 Subject: [PATCH 05/59] Simple case works --- .../Transforms/FillingTransform.cpp | 163 +++++++++++++++--- ...2730_with_fill_by_sorting_prefix.reference | 23 +++ .../02730_with_fill_by_sorting_prefix.sql | 13 ++ 3 files changed, 179 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference create mode 100644 tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 93dca03e4ac..7e9085afeb1 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -255,9 +255,10 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - input_positions.emplace_back(idx, p->second); + input_positions.emplace_back(idx, p->second); - if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.contains(column.name))) + if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.contains(column.name)) + && sort_prefix_positions.end() == std::find(sort_prefix_positions.begin(), sort_prefix_positions.end(), idx)) other_column_positions.push_back(idx); ++idx; @@ -359,6 +360,8 @@ static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, c static void copyRowFromColumns(const MutableColumnRawPtrs & dest, const Columns & source, size_t row_num) { + chassert(dest.size() == source.size()); + for (size_t i = 0, size = source.size(); i < size; ++i) dest[i]->insertFrom(*source[i], row_num); } @@ -381,12 +384,12 @@ void FillingTransform::initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, - Columns & ,//input_sort_prefix_columns, + Columns & input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, - MutableColumnRawPtrs & , //output_sort_prefix_columns, + MutableColumnRawPtrs & output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns) { Columns non_const_columns; @@ -401,7 +404,7 @@ void FillingTransform::initColumns( initColumnsByPositions(non_const_columns, input_fill_columns, output_columns, output_fill_columns, fill_column_positions); initColumnsByPositions( non_const_columns, input_interpolate_columns, output_columns, output_interpolate_columns, interpolate_column_positions); - // initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); + initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); initColumnsByPositions(non_const_columns, input_other_columns, output_columns, output_other_columns, other_column_positions); } @@ -449,31 +452,84 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut if (first) filling_row.initFromDefaults(); + /// if any rows was processed and there is sort prefix, get last row sort prefix + Columns last_row_sort_prefix; + if (!last_row.empty()) + { + last_row_sort_prefix.reserve(sort_prefix_positions.size()); + for (const size_t pos : sort_prefix_positions) + last_row_sort_prefix.push_back(last_row[pos]); + } + Block interpolate_block; if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + /// fullfill sort prefix columns with last row values or defaults + if (!last_row_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + else + for (auto * sort_prefix_column : res_sort_prefix_columns) + sort_prefix_column->insertDefault(); } while (filling_row.next(next_row)) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + /// fullfill sort prefix columns with last row values or defaults + if (!last_row_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + else + for (auto * sort_prefix_column : res_sort_prefix_columns) + sort_prefix_column->insertDefault(); } return true; } +template +size_t getRangeEnd(size_t begin, size_t end, Predicate pred) +{ + chassert(begin < end); + + const size_t linear_probe_threadhold = 16; + size_t linear_probe_end = begin + linear_probe_threadhold; + if (linear_probe_end > end) + linear_probe_end = end; + + for (size_t pos = begin; pos < linear_probe_end; ++pos) + { + if (!pred(begin, pos)) + return pos; + } + + size_t low = linear_probe_end; + size_t high = end - 1; + while (low <= high) + { + size_t mid = low + (high - low) / 2; + if (pred(begin, mid)) + low = mid + 1; + else + { + high = mid - 1; + end = mid; + } + } + return end; +} + void FillingTransform::transformRange( const Columns & input_fill_columns, const Columns & input_interpolate_columns, - const Columns &, //input_sort_prefix_columns, + const Columns & input_sort_prefix_columns, const Columns & input_other_columns, const MutableColumns & result_columns, const MutableColumnRawPtrs & res_fill_columns, const MutableColumnRawPtrs & res_interpolate_columns, - const MutableColumnRawPtrs &, //res_sort_prefix_columns, + const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, std::pair range) { @@ -485,7 +541,7 @@ void FillingTransform::transformRange( { for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[0]; + auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; if (!fill_from.isNull() && !equals(current_value, fill_from)) @@ -495,6 +551,7 @@ void FillingTransform::transformRange( { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, range_begin); } break; } @@ -530,17 +587,19 @@ void FillingTransform::transformRange( { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } while (filling_row.next(next_row)) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); - // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } } @@ -599,17 +658,81 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - transformRange( - input_fill_columns, - input_interpolate_columns, - input_sort_prefix_columns, - input_other_columns, - result_columns, - res_fill_columns, - res_interpolate_columns, - res_sort_prefix_columns, - res_other_columns, - {0, num_rows}); + if (sort_prefix.empty()) + { + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {0, num_rows}); + + saveLastRow(result_columns); + size_t num_output_rows = result_columns[0]->size(); + chunk.setColumns(std::move(result_columns), num_output_rows); + return; + } + + /// check if last row in prev chunk had the same sorting prefix as the first in new one + /// if not, we need to reinitialize filling row + if (!last_row.empty()) + { + ColumnRawPtrs last_sort_prefix_columns; + last_sort_prefix_columns.reserve(sort_prefix.size()); + for (size_t pos : sort_prefix_positions) + last_sort_prefix_columns.push_back(last_row[pos].get()); + + first = false; + for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) + { + const int res = input_sort_prefix_columns[i]->compareAt(0, 0, *last_sort_prefix_columns[i], sort_prefix[i].nulls_direction); + if (res != 0) + { + first = true; + break; + } + } + } + + for (size_t row_ind = 0; row_ind < num_rows;) + { + /// find next range + auto current_sort_prefix_end_pos = getRangeEnd( + row_ind, + num_rows, + [&](size_t pos_with_current_sort_prefix, size_t row_pos) + { + for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) + { + const int res = input_sort_prefix_columns[i]->compareAt( + pos_with_current_sort_prefix, row_pos, *input_sort_prefix_columns[i], sort_prefix[i].nulls_direction); + if (res != 0) + return false; + } + return true; + }); + + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {row_ind, current_sort_prefix_end_pos}); + + logDebug("range end", current_sort_prefix_end_pos); + row_ind = current_sort_prefix_end_pos; + first = true; + } saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference new file mode 100644 index 00000000000..121835a80e8 --- /dev/null +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -0,0 +1,23 @@ +-- { echoOn } +-- corner case with constant sort prefix +SELECT number +FROM numbers(1) +ORDER BY 10 ASC, number DESC WITH FILL FROM 1 +SETTINGS enable_positional_arguments=0; +1 +0 +-- sensor table +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); +-- FillingTransform: 6 rows will be processed in 1 chunks +select * from ts order by sensor_id, timestamp with fill step 1; +1 10 1 +1 11 0 +1 12 2 +3 5 1 +3 6 0 +3 7 3 +5 1 1 +5 2 0 +5 3 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql new file mode 100644 index 00000000000..828572eb620 --- /dev/null +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -0,0 +1,13 @@ +-- { echoOn } +-- corner case with constant sort prefix +SELECT number +FROM numbers(1) +ORDER BY 10 ASC, number DESC WITH FILL FROM 1 +SETTINGS enable_positional_arguments=0; + +-- sensor table +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); +-- FillingTransform: 6 rows will be processed in 1 chunks +select * from ts order by sensor_id, timestamp with fill step 1; From 690ebb9d6bdc3a97eb00f6538ece8503108d9d1c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 11:33:53 +0000 Subject: [PATCH 06/59] Fast fix: correct suffix generation --- src/Processors/Transforms/FillingTransform.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 7e9085afeb1..6bd4557e74b 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -414,6 +414,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut logDebug("generateSuffixIfNeeded() next_row", next_row); logDebug("generateSuffixIfNeeded() first", first); + first = last_row.empty(); /// Determines should we insert filling row before start generating next rows. bool should_insert_first = next_row < filling_row || first; @@ -588,6 +589,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + logDebug("filling_row should_insert_first", filling_row); } while (filling_row.next(next_row)) @@ -595,6 +597,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + logDebug("filling_row", filling_row); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); From 80bf023382089d30112287e665125fa0dfd7face Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 12:18:20 +0000 Subject: [PATCH 07/59] Fix typos --- src/Processors/Transforms/FillingTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 6bd4557e74b..27090a4078c 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -467,7 +467,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - /// fullfill sort prefix columns with last row values or defaults + /// fulfill sort prefix columns with last row values or defaults if (!last_row_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); else @@ -479,7 +479,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - /// fullfill sort prefix columns with last row values or defaults + /// fulfill sort prefix columns with last row values or defaults if (!last_row_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); else From 5200a31da33a29814a57bac728f7e65596a4d147 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 20:07:25 +0000 Subject: [PATCH 08/59] Check if processing range with new sort prefix differently --- .../Transforms/FillingTransform.cpp | 32 +++++++++++-------- src/Processors/Transforms/FillingTransform.h | 4 +-- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 27090a4078c..29bc5367774 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -410,20 +410,22 @@ void FillingTransform::initColumns( bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns) { + /// true means we'll generate rows for empty result set + const bool no_data_processed = last_row.empty(); + logDebug("generateSuffixIfNeeded() filling_row", filling_row); logDebug("generateSuffixIfNeeded() next_row", next_row); - logDebug("generateSuffixIfNeeded() first", first); + logDebug("generateSuffixIfNeeded() no_data_processed", no_data_processed); - first = last_row.empty(); /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || first; + bool should_insert_first = next_row < filling_row || no_data_processed; for (size_t i = 0, size = filling_row.size(); i < size; ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; logDebug("generateSuffixIfNeeded() next_row updated", next_row); - if (!first && filling_row >= next_row) + if (!no_data_processed && filling_row >= next_row) { logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); return false; @@ -450,7 +452,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut res_sort_prefix_columns, res_other_columns); - if (first) + if (no_data_processed) filling_row.initFromDefaults(); /// if any rows was processed and there is sort prefix, get last row sort prefix @@ -532,13 +534,14 @@ void FillingTransform::transformRange( const MutableColumnRawPtrs & res_interpolate_columns, const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, - std::pair range) + std::pair range, + const bool new_sorting_prefix) { const size_t range_begin = range.first; const size_t range_end = range.second; Block interpolate_block; - if (first) + if (new_sorting_prefix) { for (size_t i = 0, size = filling_row.size(); i < size; ++i) { @@ -558,7 +561,6 @@ void FillingTransform::transformRange( } filling_row[i] = current_value; } - first = false; } for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) @@ -663,6 +665,7 @@ void FillingTransform::transform(Chunk & chunk) if (sort_prefix.empty()) { + constexpr bool new_sort_prefix = true; transformRange( input_fill_columns, input_interpolate_columns, @@ -673,7 +676,8 @@ void FillingTransform::transform(Chunk & chunk) res_interpolate_columns, res_sort_prefix_columns, res_other_columns, - {0, num_rows}); + {0, num_rows}, + new_sort_prefix); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); @@ -683,6 +687,7 @@ void FillingTransform::transform(Chunk & chunk) /// check if last row in prev chunk had the same sorting prefix as the first in new one /// if not, we need to reinitialize filling row + bool new_sort_prefix = last_row.empty(); if (!last_row.empty()) { ColumnRawPtrs last_sort_prefix_columns; @@ -690,13 +695,13 @@ void FillingTransform::transform(Chunk & chunk) for (size_t pos : sort_prefix_positions) last_sort_prefix_columns.push_back(last_row[pos].get()); - first = false; + new_sort_prefix = false; for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) { const int res = input_sort_prefix_columns[i]->compareAt(0, 0, *last_sort_prefix_columns[i], sort_prefix[i].nulls_direction); if (res != 0) { - first = true; + new_sort_prefix = true; break; } } @@ -730,11 +735,12 @@ void FillingTransform::transform(Chunk & chunk) res_interpolate_columns, res_sort_prefix_columns, res_other_columns, - {row_ind, current_sort_prefix_end_pos}); + {row_ind, current_sort_prefix_end_pos}, + new_sort_prefix); logDebug("range end", current_sort_prefix_end_pos); row_ind = current_sort_prefix_end_pos; - first = true; + new_sort_prefix = true; } saveLastRow(result_columns); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index dac92e43231..0506bb92efb 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -43,7 +43,8 @@ private: const MutableColumnRawPtrs & res_interpolate_columns, const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, - std::pair range); + std::pair range, + bool new_sorting_prefix); void saveLastRow(const MutableColumns & result_columns); void interpolate(const MutableColumns & result_columns, Block & interpolate_block); @@ -80,7 +81,6 @@ private: std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; - bool first = true; /// flag to determine if transform is/will be called for the first time bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks }; From 0dfeb880ceb04e6a8e4043baaa1745b3308ff76d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 7 May 2023 08:08:02 +0000 Subject: [PATCH 09/59] Fix 01034_with_fill_and_push_down_predicate.sql --- src/Processors/Transforms/FillingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 29bc5367774..b9c3341948a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -665,7 +665,6 @@ void FillingTransform::transform(Chunk & chunk) if (sort_prefix.empty()) { - constexpr bool new_sort_prefix = true; transformRange( input_fill_columns, input_interpolate_columns, @@ -677,7 +676,7 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns, {0, num_rows}, - new_sort_prefix); + last_row.empty()); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); From 93161f70d15e4a3587ddde494532a874c6b432d0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 10:45:43 +0000 Subject: [PATCH 10/59] Correct suffix handling for ranges --- .../Transforms/FillingTransform.cpp | 84 +++++++++++-------- src/Processors/Transforms/FillingTransform.h | 9 +- ...2730_with_fill_by_sorting_prefix.reference | 51 +++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 19 +++++ 4 files changed, 126 insertions(+), 37 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index b9c3341948a..5fa89e89a4b 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -246,6 +246,7 @@ FillingTransform::FillingTransform( sort_prefix.push_back(desc); } logDebug("sort prefix", dumpSortDescription(sort_prefix)); + last_range_sort_prefix.reserve(sort_prefix.size()); /// TODO: check conflict in positions between interpolate and sorting prefix columns @@ -334,6 +335,8 @@ using MutableColumnRawPtrs = std::vector; static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, const MutableColumnRawPtrs & interpolate_columns, const MutableColumnRawPtrs & other_columns, const FillingRow & filling_row, const Block & interpolate_block) { + logDebug("insertFromFillingRow", filling_row); + for (size_t i = 0, size = filling_columns.size(); i < size; ++i) { if (filling_row[i].isNull()) @@ -410,27 +413,6 @@ void FillingTransform::initColumns( bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns) { - /// true means we'll generate rows for empty result set - const bool no_data_processed = last_row.empty(); - - logDebug("generateSuffixIfNeeded() filling_row", filling_row); - logDebug("generateSuffixIfNeeded() next_row", next_row); - logDebug("generateSuffixIfNeeded() no_data_processed", no_data_processed); - - /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || no_data_processed; - - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - next_row[i] = filling_row.getFillDescription(i).fill_to; - - logDebug("generateSuffixIfNeeded() next_row updated", next_row); - - if (!no_data_processed && filling_row >= next_row) - { - logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); - return false; - } - Columns input_fill_columns; Columns input_interpolate_columns; Columns input_sort_prefix_columns; @@ -452,16 +434,32 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut res_sort_prefix_columns, res_other_columns); - if (no_data_processed) - filling_row.initFromDefaults(); + return generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); +} - /// if any rows was processed and there is sort prefix, get last row sort prefix - Columns last_row_sort_prefix; - if (!last_row.empty()) +bool FillingTransform::generateSuffixIfNeeded( + const MutableColumns & result_columns, + MutableColumnRawPtrs res_fill_columns, + MutableColumnRawPtrs res_interpolate_columns, + MutableColumnRawPtrs res_sort_prefix_columns, + MutableColumnRawPtrs res_other_columns) +{ + logDebug("generateSuffixIfNeeded() filling_row", filling_row); + logDebug("generateSuffixIfNeeded() next_row", next_row); + + /// Determines should we insert filling row before start generating next rows. + bool should_insert_first = next_row < filling_row; + logDebug("should_insert_first", should_insert_first); + + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; + + logDebug("generateSuffixIfNeeded() next_row updated", next_row); + + if (filling_row >= next_row) { - last_row_sort_prefix.reserve(sort_prefix_positions.size()); - for (const size_t pos : sort_prefix_positions) - last_row_sort_prefix.push_back(last_row[pos]); + logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); + return false; } Block interpolate_block; @@ -470,8 +468,8 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); /// fulfill sort prefix columns with last row values or defaults - if (!last_row_sort_prefix.empty()) - copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + if (!last_range_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); else for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); @@ -482,8 +480,8 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); /// fulfill sort prefix columns with last row values or defaults - if (!last_row_sort_prefix.empty()) - copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + if (!last_range_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); else for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); @@ -591,7 +589,6 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - logDebug("filling_row should_insert_first", filling_row); } while (filling_row.next(next_row)) @@ -599,7 +596,6 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - logDebug("filling_row", filling_row); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); @@ -607,6 +603,15 @@ void FillingTransform::transformRange( copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } + + /// save sort prefix of last row in the range, it's used to generate suffix + last_range_sort_prefix.clear(); + for (const auto & sort_prefix_column : input_sort_prefix_columns) + { + auto column = sort_prefix_column->cloneEmpty(); + column->insertFrom(*sort_prefix_column, range_end - 1); + last_range_sort_prefix.push_back(std::move(column)); + } } void FillingTransform::transform(Chunk & chunk) @@ -638,6 +643,11 @@ void FillingTransform::transform(Chunk & chunk) /// if all chunks are processed, then we may need to generate suffix for the following cases: /// (1) when all data are processed and WITH FILL .. TO is provided /// (2) for empty result set when WITH FILL FROM .. TO is provided (see PR #30888) + + /// if no data was processed, then need to initialize filling_row + if (last_row.empty()) + filling_row.initFromDefaults(); + if (generateSuffixIfNeeded(input.getHeader().getColumns(), result_columns)) { size_t num_output_rows = result_columns[0]->size(); @@ -724,6 +734,10 @@ void FillingTransform::transform(Chunk & chunk) return true; }); + /// generate suffix for the previous range + if (!last_range_sort_prefix.empty()) + generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); + transformRange( input_fill_columns, input_interpolate_columns, diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 0506bb92efb..def4e87c7ed 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -62,8 +62,12 @@ private: MutableColumnRawPtrs & output_other_columns); bool generateSuffixIfNeeded( - const Columns & input_columns, - MutableColumns & result_columns); + const MutableColumns & result_columns, + MutableColumnRawPtrs res_fill_columns, + MutableColumnRawPtrs res_interpolate_columns, + MutableColumnRawPtrs res_sort_prefix_columns, + MutableColumnRawPtrs res_other_columns); + bool generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns); const SortDescription sort_description; const SortDescription fill_description; /// Contains only columns with WITH FILL. @@ -81,6 +85,7 @@ private: std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; + Columns last_range_sort_prefix; bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks }; diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 121835a80e8..54a4faf46f2 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -21,3 +21,54 @@ select * from ts order by sensor_id, timestamp with fill step 1; 5 1 1 5 2 0 5 3 1 +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +1 10 1 +1 11 0 +1 12 1 +3 5 1 +3 6 0 +3 7 1 +5 1 1 +5 2 0 +5 3 1 +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +1 10 1 +1 11 0 +1 12 1 +3 5 1 +3 6 0 +3 7 1 +5 1 1 +5 2 0 +5 3 1 +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +5 1 1 +5 3 1 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 828572eb620..1b2288da323 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -11,3 +11,22 @@ create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=Merge insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); -- FillingTransform: 6 rows will be processed in 1 chunks select * from ts order by sensor_id, timestamp with fill step 1; + +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; + +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; + +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); From a0ef2e272faf5c4f2ab1dd74274ab624b74e7f16 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 12:41:33 +0000 Subject: [PATCH 11/59] Fix suffix generation for empty result set --- src/Interpreters/FillingRow.cpp | 9 +++++++++ src/Interpreters/FillingRow.h | 1 + src/Processors/Transforms/FillingTransform.cpp | 2 +- 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 98197533aa1..87cc91681c4 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -50,6 +50,15 @@ bool FillingRow::operator>=(const FillingRow & other) const return !(*this < other); } +bool FillingRow::isNull() const +{ + for (const auto & field : row) + if (!field.isNull()) + return false; + + return true; +} + bool FillingRow::next(const FillingRow & to_row) { const size_t row_size = size(); diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 9f414d92dd3..b1af851d792 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -29,6 +29,7 @@ public: bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; bool operator>=(const FillingRow & other) const; + bool isNull() const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 5fa89e89a4b..45967b646a6 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -448,7 +448,7 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded() next_row", next_row); /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row; + bool should_insert_first = next_row < filling_row || next_row.isNull(); logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) From 5143b37d0612fe1cc00f3fc91a7ca13423e851fa Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 17:03:07 +0000 Subject: [PATCH 12/59] Fix clang-tidy build --- src/Planner/Planner.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e3a4d01a46a..e19a92c273c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -623,7 +623,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, } auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_description); + query_plan.getCurrentDataStream(), sort_description, std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } From 72f4aed64b8e8367ef4ab5ffae8ef67c58a09f15 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 21:08:29 +0000 Subject: [PATCH 13/59] Add setting use_with_fill_by_sorting_prefix --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++++- src/Planner/Planner.cpp | 8 ++++++- src/Processors/QueryPlan/FillingStep.cpp | 7 +++++-- src/Processors/QueryPlan/FillingStep.h | 4 +++- .../Transforms/FillingTransform.cpp | 21 ++++++++++++------- src/Processors/Transforms/FillingTransform.h | 4 +++- 7 files changed, 39 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1bea2c26392..0d7ffbadf8c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -716,6 +716,7 @@ class IColumn; \ M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \ \ + M(Bool, use_with_fill_by_sorting_prefix, false, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with the same values in sorting prefix will be filled separately", 0) \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e36a1287503..f6da77d66ed 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3008,8 +3008,14 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + + const Settings & settings = context->getSettingsRef(); auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_descr); + query_plan.getCurrentDataStream(), + std::move(sort_description), + std::move(fill_description), + interpolate_descr, + settings.use_with_fill_by_sorting_prefix); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e19a92c273c..ba338de4072 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -622,8 +622,14 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } + const auto & query_context = planner_context->getQueryContext(); + const Settings & settings = query_context->getSettingsRef(); auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), sort_description, std::move(fill_description), interpolate_description); + query_plan.getCurrentDataStream(), + sort_description, + std::move(fill_description), + interpolate_description, + settings.use_with_fill_by_sorting_prefix); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 37c5307fcce..65c9cf11661 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -31,11 +31,13 @@ FillingStep::FillingStep( const DataStream & input_stream_, SortDescription sort_description_, SortDescription fill_description_, - InterpolateDescriptionPtr interpolate_description_) + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix_) : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) , sort_description(std::move(sort_description_)) , fill_description(std::move(fill_description_)) , interpolate_description(interpolate_description_) + , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (!input_stream_.has_single_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); @@ -48,7 +50,8 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (stream_type == QueryPipelineBuilder::StreamType::Totals) return std::make_shared(header, fill_description); - return std::make_shared(header, sort_description, fill_description, std::move(interpolate_description)); + return std::make_shared( + header, sort_description, fill_description, std::move(interpolate_description), use_with_fill_by_sorting_prefix); }); } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 0febbbb8eb0..3e0e1e79060 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -14,7 +14,8 @@ public: const DataStream & input_stream_, SortDescription sort_description_, SortDescription fill_description_, - InterpolateDescriptionPtr interpolate_description_); + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix); String getName() const override { return "Filling"; } @@ -31,6 +32,7 @@ private: SortDescription sort_description; SortDescription fill_description; InterpolateDescriptionPtr interpolate_description; + const bool use_with_fill_by_sorting_prefix; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 45967b646a6..5b3200ac89e 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -187,13 +187,18 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) - : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) - , sort_description(sort_description_) - , fill_description(fill_description_) - , interpolate_description(interpolate_description_) - , filling_row(fill_description_) - , next_row(fill_description_) + const Block & header_, + const SortDescription & sort_description_, + const SortDescription & fill_description_, + InterpolateDescriptionPtr interpolate_description_, + const bool use_with_fill_by_sorting_prefix_) + : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) + , sort_description(sort_description_) + , fill_description(fill_description_) + , interpolate_description(interpolate_description_) + , filling_row(fill_description_) + , next_row(fill_description_) + , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) interpolate_actions = std::make_shared(interpolate_description->actions); @@ -673,7 +678,7 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - if (sort_prefix.empty()) + if (sort_prefix.empty() || !use_with_fill_by_sorting_prefix) { transformRange( input_fill_columns, diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index def4e87c7ed..2a4da05ae29 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -20,7 +20,8 @@ public: const Block & header_, const SortDescription & sort_description_, const SortDescription & fill_description_, - InterpolateDescriptionPtr interpolate_description_); + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix_); String getName() const override { return "FillingTransform"; } @@ -87,6 +88,7 @@ private: Columns last_row; Columns last_range_sort_prefix; bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks + const bool use_with_fill_by_sorting_prefix; }; class FillingNoopTransform : public ISimpleTransform From 3355894631fea5cbe05851c2bad4e4b36f94bfca Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 13 May 2023 21:43:27 +0000 Subject: [PATCH 14/59] Update test (found bug in previous implementation) --- .../02730_with_fill_by_sorting_prefix.reference | 12 ++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 3 +++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 54a4faf46f2..26cf5200e87 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -1,4 +1,5 @@ -- { echoOn } +set use_with_fill_by_sorting_prefix=1; -- corner case with constant sort prefix SELECT number FROM numbers(1) @@ -72,3 +73,14 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 5 7 9999 5 8 9999 5 9 9999 +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 1b2288da323..3363dae35b8 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -1,4 +1,6 @@ -- { echoOn } +set use_with_fill_by_sorting_prefix=1; + -- corner case with constant sort prefix SELECT number FROM numbers(1) @@ -30,3 +32,4 @@ insert into ts VALUES (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; From 629a1a6668f8dc4a25106e72745cf6048181fc01 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 13 May 2023 22:11:23 +0000 Subject: [PATCH 15/59] Minor cleanup --- src/Interpreters/FillingRow.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 87cc91681c4..3ee3b681280 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -101,8 +101,7 @@ bool FillingRow::next(const FillingRow & to_row) if (equals(row[pos], to_row.row[pos])) { bool is_less = false; - size_t i = pos + 1; - for (; i < row_size; ++i) + for (size_t i = pos + 1; i < row_size; ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) From 93384d6546ef0d7a5a023b40f7f4d2e411f1faea Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 May 2023 15:52:49 +0000 Subject: [PATCH 16/59] Fix suffix generation (see added test in commit 3355894) --- src/Interpreters/FillingRow.cpp | 12 ++-- src/Interpreters/FillingRow.h | 5 +- .../Transforms/FillingTransform.cpp | 61 ++++++++++++++----- src/Processors/Transforms/FillingTransform.h | 7 +++ 4 files changed, 63 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 3ee3b681280..7efbb62b0d2 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -59,7 +59,7 @@ bool FillingRow::isNull() const return true; } -bool FillingRow::next(const FillingRow & to_row) +std::pair FillingRow::next(const FillingRow & to_row) { const size_t row_size = size(); size_t pos = 0; @@ -70,7 +70,7 @@ bool FillingRow::next(const FillingRow & to_row) break; if (pos == row_size || less(to_row.row[pos], row[pos], getDirection(pos))) - return false; + return {false, false}; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. @@ -87,7 +87,7 @@ bool FillingRow::next(const FillingRow & to_row) { row[i] = next_value; initFromDefaults(i + 1); - return true; + return {true, true}; } } @@ -95,7 +95,7 @@ bool FillingRow::next(const FillingRow & to_row) getFillDescription(pos).step_func(next_value); if (less(to_row.row[pos], next_value, getDirection(pos)) || equals(next_value, getFillDescription(pos).fill_to)) - return false; + return {false, false}; row[pos] = next_value; if (equals(row[pos], to_row.row[pos])) @@ -111,11 +111,11 @@ bool FillingRow::next(const FillingRow & to_row) is_less |= less(row[i], to_row.row[i], getDirection(i)); } - return is_less; + return {is_less, true}; } initFromDefaults(pos + 1); - return true; + return {true, true}; } void FillingRow::initFromDefaults(size_t from_pos) diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index b1af851d792..004b417542c 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -19,7 +19,10 @@ public: explicit FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. - bool next(const FillingRow & to_row); + /// Return pair of boolean + /// apply - true if filling values should be inserted into result set + /// value_changed - true if filling row value was changed + std::pair next(const FillingRow & to_row); void initFromDefaults(size_t from_pos = 0); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 5b3200ac89e..e4bd689471a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -337,8 +337,11 @@ void FillingTransform::interpolate(const MutableColumns & result_columns, Block using MutableColumnRawPtrs = std::vector; -static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, const MutableColumnRawPtrs & interpolate_columns, const MutableColumnRawPtrs & other_columns, - const FillingRow & filling_row, const Block & interpolate_block) +void FillingTransform::insertFromFillingRow( + const MutableColumnRawPtrs & filling_columns, + const MutableColumnRawPtrs & interpolate_columns, + const MutableColumnRawPtrs & other_columns, + const Block & interpolate_block) { logDebug("insertFromFillingRow", filling_row); @@ -364,6 +367,8 @@ static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, c for (auto * other_column : other_columns) other_column->insertDefault(); + + filling_row_inserted = true; } static void copyRowFromColumns(const MutableColumnRawPtrs & dest, const Columns & source, size_t row_num) @@ -452,8 +457,8 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded() filling_row", filling_row); logDebug("generateSuffixIfNeeded() next_row", next_row); - /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || next_row.isNull(); + /// Determines if we should insert filling row before start generating next rows + bool should_insert_first = (next_row < filling_row && !filling_row_inserted) || next_row.isNull(); logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) @@ -471,7 +476,7 @@ bool FillingTransform::generateSuffixIfNeeded( if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); /// fulfill sort prefix columns with last row values or defaults if (!last_range_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); @@ -480,10 +485,16 @@ bool FillingTransform::generateSuffixIfNeeded( sort_prefix_column->insertDefault(); } - while (filling_row.next(next_row)) + bool filling_row_changed = false; + while (true) { + const auto [apply, changed] = filling_row.next(next_row); + filling_row_changed = changed; + if (!apply) + break; + interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); /// fulfill sort prefix columns with last row values or defaults if (!last_range_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); @@ -491,6 +502,9 @@ bool FillingTransform::generateSuffixIfNeeded( for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); } + /// new valid filling row was generated but not inserted + if (filling_row_changed) + filling_row_inserted = false; return true; } @@ -546,18 +560,20 @@ void FillingTransform::transformRange( Block interpolate_block; if (new_sorting_prefix) { + logDebug("--- new range ---", range_end); for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[range_begin]; + const auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; if (!fill_from.isNull() && !equals(current_value, fill_from)) { filling_row.initFromDefaults(i); + filling_row_inserted = false; if (less(fill_from, current_value, filling_row.getDirection(i))) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, range_begin); } break; @@ -577,7 +593,7 @@ void FillingTransform::transformRange( for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[row_ind]; + const auto current_value = (*input_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) @@ -587,21 +603,33 @@ void FillingTransform::transformRange( } logDebug("next_row updated", next_row); - /// A case, when at previous step row was initialized from defaults 'fill_from' values - /// and probably we need to insert it to block. + /// The condition is true when filling row is initialized by value(s) in FILL FROM, + /// and there are row(s) in current range with value(s) < then in the filling row. + /// It can happen only once for a range. if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } - while (filling_row.next(next_row)) + bool filling_row_changed = false; + while(true) { + const auto [apply, changed] = filling_row.next(next_row); + filling_row_changed = changed; + if (!apply) + break; + interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } + /// new valid filling row was generated but not inserted, will use it during suffix generation + if (filling_row_changed) + filling_row_inserted = false; + + logDebug("filling_row after", filling_row); copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); @@ -651,7 +679,10 @@ void FillingTransform::transform(Chunk & chunk) /// if no data was processed, then need to initialize filling_row if (last_row.empty()) + { filling_row.initFromDefaults(); + filling_row_inserted = false; + } if (generateSuffixIfNeeded(input.getHeader().getColumns(), result_columns)) { diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 2a4da05ae29..85da544ef2d 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -70,6 +70,12 @@ private: MutableColumnRawPtrs res_other_columns); bool generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns); + void insertFromFillingRow( + const MutableColumnRawPtrs & filling_columns, + const MutableColumnRawPtrs & interpolate_columns, + const MutableColumnRawPtrs & other_columns, + const Block & interpolate_block); + const SortDescription sort_description; const SortDescription fill_description; /// Contains only columns with WITH FILL. SortDescription sort_prefix; @@ -77,6 +83,7 @@ private: FillingRow filling_row; /// Current row, which is used to fill gaps. FillingRow next_row; /// Row to which we need to generate filling rows. + bool filling_row_inserted = false; using Positions = std::vector; Positions fill_column_positions; From b2dddf7cf89e9b31c88b58b4cb78542432d71773 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 May 2023 16:06:35 +0000 Subject: [PATCH 17/59] Fix style --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index e4bd689471a..93bec15bae7 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -614,7 +614,7 @@ void FillingTransform::transformRange( } bool filling_row_changed = false; - while(true) + while (true) { const auto [apply, changed] = filling_row.next(next_row); filling_row_changed = changed; From 0d585746de23feee1b323b850f6c76331006e5c0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 15 May 2023 17:08:00 +0000 Subject: [PATCH 18/59] Fixes + disallow usage of the same column in sorting prefix and INTERPOLATE + sorting prefix is empty if use_with_fill_by_sorting_prefix is off --- .../Transforms/FillingTransform.cpp | 37 +++++++++++++------ ...2730_with_fill_by_sorting_prefix.reference | 10 +++-- .../02730_with_fill_by_sorting_prefix.sql | 3 ++ 3 files changed, 35 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 93bec15bae7..052a5a1c183 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -239,21 +239,22 @@ FillingTransform::FillingTransform( if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); - /// build sorting prefix for first fill column - for (const auto & desc : sort_description) + if (use_with_fill_by_sorting_prefix) { - if (desc.column_name == fill_description[0].column_name) - break; + /// build sorting prefix for first fill column + for (const auto & desc : sort_description) + { + if (desc.column_name == fill_description[0].column_name) + break; - size_t pos = header_.getPositionByName(desc.column_name); - sort_prefix_positions.push_back(pos); + size_t pos = header_.getPositionByName(desc.column_name); + sort_prefix_positions.push_back(pos); - sort_prefix.push_back(desc); + sort_prefix.push_back(desc); + } + logDebug("sort prefix", dumpSortDescription(sort_prefix)); + last_range_sort_prefix.reserve(sort_prefix.size()); } - logDebug("sort prefix", dumpSortDescription(sort_prefix)); - last_range_sort_prefix.reserve(sort_prefix.size()); - - /// TODO: check conflict in positions between interpolate and sorting prefix columns size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) @@ -273,6 +274,20 @@ FillingTransform::FillingTransform( if (interpolate_description) for (const auto & name : interpolate_description->result_columns_order) interpolate_column_positions.push_back(header_.getPositionByName(name)); + + /// check conflict in positions between interpolate and sorting prefix columns + if (!sort_prefix_positions.empty() && !interpolate_column_positions.empty()) + { + std::unordered_set interpolate_positions(interpolate_column_positions.begin(), interpolate_column_positions.end()); + for (auto sort_prefix_pos : sort_prefix_positions) + { + if (interpolate_positions.contains(sort_prefix_pos)) + throw Exception( + ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "The same column in ORDER BY before WITH FILL (sorting prefix) and INTERPOLATE is not allowed. Column: {}", + (header_.begin() + sort_prefix_pos)->name); + } + } } /// prepare() is overrididen to call transform() after all chunks are processed diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 26cf5200e87..3e2f939af28 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -74,13 +74,15 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 5 8 9999 5 9 9999 select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -1 6 9999 -1 7 9999 -1 8 9999 -1 9 9999 +0 6 9999 +0 7 9999 +0 8 9999 +0 9 9999 1 10 1 1 12 1 3 5 1 3 7 1 5 1 1 5 3 1 +-- checking that sorting prefix columns can't be used in INTERPOLATE +SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 3363dae35b8..01fc125672a 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -33,3 +33,6 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + +-- checking that sorting prefix columns can't be used in INTERPOLATE +SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From df436b2cd4a7f93bffc8e0295f5cbb3b1ac7b3c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 06:59:08 +0000 Subject: [PATCH 19/59] Spark compatibility: Add new function space() --- .../functions/string-functions.md | 42 ++++- src/Functions/repeat.cpp | 41 ++--- src/Functions/space.cpp | 166 ++++++++++++++++++ .../02752_space_function.reference | 51 ++++++ .../0_stateless/02752_space_function.sql | 50 ++++++ 5 files changed, 328 insertions(+), 22 deletions(-) create mode 100644 src/Functions/space.cpp create mode 100644 tests/queries/0_stateless/02752_space_function.reference create mode 100644 tests/queries/0_stateless/02752_space_function.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 920a1f972d8..8662d08431c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -323,11 +323,11 @@ Alias: `REPEAT` **Arguments** - `s` — The string to repeat. [String](../../sql-reference/data-types/string.md). -- `n` — The number of times to repeat the string. [UInt or Int](../../sql-reference/data-types/int-uint.md). +- `n` — The number of times to repeat the string. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). **Returned value** -The single string containing string `s` repeated `n` times. If `n` \< 1, the function returns empty string. +A string containing string `s` repeated `n` times. If `n` <= 0, the function returns the empty string. Type: `String`. @@ -345,6 +345,44 @@ Result: └────────────────────────────────┘ ``` +## space + +Concatenates a space (` `) as many times with itself as specified. + +**Syntax** + +``` sql +space(n) +``` + +Alias: `SPACE`. + +**Arguments** + +- `n` — The number of times to repeat the space. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +The string containing string ` ` repeated `n` times. If `n` <= 0, the function returns the empty string. + +Type: `String`. + +**Example** + +Query: + +``` sql +SELECT space(3); +``` + +Result: + +``` text +┌─space(3) ────┐ +│ │ +└──────────────┘ +``` + ## reverse Reverses the sequence of bytes in a string. diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 0c323c39969..88eaebf4f03 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -186,36 +186,37 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - if (!isInteger(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[1]->getName(), getName()); - return arguments[0]; + FunctionArgumentDescriptors args{ + {"s", &isString, nullptr, "String"}, + {"n", &isInteger, nullptr, "Integer"}, + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override { - const auto & strcolumn = arguments[0].column; - const auto & numcolumn = arguments[1].column; + const auto & col_str = arguments[0].column; + const auto & col_num = arguments[1].column; ColumnPtr res; - if (const ColumnString * col = checkAndGetColumn(strcolumn.get())) + if (const ColumnString * col = checkAndGetColumn(col_str.get())) { - if (const ColumnConst * scale_column_num = checkAndGetColumn(numcolumn.get())) + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) { auto col_res = ColumnString::create(); castType(arguments[1].type.get(), [&](const auto & type) { using DataType = std::decay_t; using T = typename DataType::FieldType; - T repeat_time = scale_column_num->getValue(); - RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); + T times = col_num_const->getValue(); + RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), times); return true; }); return col_res; @@ -224,9 +225,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) @@ -234,7 +235,7 @@ public: return res; } } - else if (const ColumnConst * col_const = checkAndGetColumn(strcolumn.get())) + else if (const ColumnConst * col_const = checkAndGetColumn(col_str.get())) { /// Note that const-const case is handled by useDefaultImplementationForConstants. @@ -244,9 +245,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp new file mode 100644 index 00000000000..afdc01b717d --- /dev/null +++ b/src/Functions/space.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +/// Prints whitespace n-times. Actually, space() could also be pushed down to repeat(). Chose a standalone-implementation because +/// we can do memset() whereas repeat() does memcpy(). +class FunctionSpace : public IFunction +{ +private: + static constexpr auto space = ' '; + +public: + static constexpr auto name = "space"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"n", &isInteger, nullptr, "Integer"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + + template + bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + const ColumnConst * col_times_const = checkAndGetColumn(col_times.get()); + + const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr(); + if (!checkAndGetColumn(col_times_const_internal.get())) + return false; + + using T = typename DataType::FieldType; + T times = col_times_const->getValue(); + + if (times < 1) + times = 0; + + res_offsets.resize(col_times->size()); + res_chars.resize(col_times->size() * (times + 1)); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + return true; + } + + + template + bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + auto * col_times = checkAndGetColumn(col_times_.get()); + if (!col_times) + return false; + + res_offsets.resize(col_times->size()); + res_chars.reserve(col_times->size() * 10); /// heuristic + + const PaddedPODArray & times_data = col_times->getData(); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + typename DataType::FieldType times = times_data[i]; + + if (times < 1) + times = 0; + + if (res_chars.size() + times + 1 >= res_chars.capacity()) + res_chars.reserve(2 * res_chars.capacity()); + + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + res_chars.resize(pos); + + return true; + } + + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const auto & col_num = arguments[0].column; + + auto col_res = ColumnString::create(); + + ColumnString::Offsets & res_offsets = col_res->getOffsets(); + ColumnString::Chars & res_chars = col_res->getChars(); + + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) + { + if ((executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars))) + return col_res; + } + else + { + if ((executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars))) + return col_res; + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; +} + +REGISTER_FUNCTION(Space) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference new file mode 100644 index 00000000000..b0adccb415b --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -0,0 +1,51 @@ +const times, uint + + + + +const times, int + + + + +negative const times + + + + +negative tests +\N +non-const times, uint + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 4 + 21 + 9 + 7 + 56 + 20 + 5 + 7 +non-const times, int + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 0 + 0 + 0 + 0 + 56 + 20 + 5 + 7 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql new file mode 100644 index 00000000000..8f3605e6e6f --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -0,0 +1,50 @@ +SELECT 'const times, uint'; +SELECT space(3::UInt8); +SELECT space(3::UInt16); +SELECT space(3::UInt32); +SELECT space(3::UInt64); +SELECT 'const times, int'; +SELECT space(3::Int8); +SELECT space(3::Int16); +SELECT space(3::Int32); +SELECT space(3::Int64); + +SELECT 'negative const times'; +SELECT space(-3::Int8); +SELECT space(-3::Int16); +SELECT space(-3::Int32); +SELECT space(-3::Int64); + +SELECT 'negative tests'; +SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(NULL); + +DROP TABLE IF EXISTS defaults; +CREATE TABLE defaults +( + u8 UInt8, + u16 UInt16, + u32 UInt32, + u64 UInt64, + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64 +) ENGINE = Memory(); + +INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); + +SELECT 'non-const times, uint'; +SELECT space(u8), length(space(u8)) FROM defaults; +SELECT space(u16), length(space(u16)) FROM defaults; +SELECT space(u32), length(space(u32)) from defaults; +SELECT space(u64), length(space(u64)) FROM defaults; +SELECT 'non-const times, int'; +SELECT space(i8), length(space(i8)) FROM defaults; +SELECT space(i16), length(space(i16)) FROM defaults; +SELECT space(i32), length(space(i32)) from defaults; +SELECT space(i64), length(space(i64)) FROM defaults; + +DROP TABLE defaults; From c4d862a16fa9d66c7c1fc3e67a615c15b9e37a60 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 May 2023 19:51:58 +0200 Subject: [PATCH 20/59] Make async reader work with any impl --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 1 + src/Backups/BackupEntryFromImmutableFile.cpp | 2 + src/Backups/BackupIO_Default.cpp | 2 +- src/Backups/BackupImpl.cpp | 2 +- src/Coordination/KeeperStateManager.cpp | 1 + src/Disks/IDisk.h | 2 + ....cpp => AsynchronousBoundedReadBuffer.cpp} | 127 ++++++------------ src/Disks/IO/AsynchronousBoundedReadBuffer.h | 96 +++++++++++++ ...ynchronousReadIndirectBufferFromRemoteFS.h | 111 --------------- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 61 +++------ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 56 +++----- .../IO/ReadIndirectBufferFromRemoteFS.cpp | 2 +- src/Disks/IO/ReadIndirectBufferFromRemoteFS.h | 2 - .../AzureBlobStorage/AzureObjectStorage.cpp | 5 +- .../AzureBlobStorage/AzureObjectStorage.h | 1 - ...ObjectStorageRemoteMetadataRestoreHelper.h | 1 + .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 - src/Disks/ObjectStorages/IObjectStorage.cpp | 1 + src/Disks/ObjectStorages/IObjectStorage.h | 4 +- .../Local/LocalObjectStorage.cpp | 7 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 6 +- src/Disks/ObjectStorages/StoredObject.cpp | 14 ++ src/Disks/ObjectStorages/StoredObject.h | 2 + .../ObjectStorages/Web/WebObjectStorage.cpp | 5 +- src/IO/AsyncReadCounters.h | 1 + src/IO/ReadBufferFromFileDecorator.h | 2 - src/IO/SeekableReadBuffer.h | 2 - .../FilesystemReadPrefetchesLog.h | 2 + .../MergeTree/MergeTreeDeduplicationLog.cpp | 1 + src/Storages/StorageS3.cpp | 6 +- src/Storages/StorageSet.cpp | 1 + 31 files changed, 230 insertions(+), 297 deletions(-) rename src/Disks/IO/{AsynchronousReadIndirectBufferFromRemoteFS.cpp => AsynchronousBoundedReadBuffer.cpp} (72%) create mode 100644 src/Disks/IO/AsynchronousBoundedReadBuffer.h delete mode 100644 src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h create mode 100644 src/Disks/ObjectStorages/StoredObject.cpp diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 1d73ab52820..5303d9abffd 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index cc635dd8541..d066db0c6ae 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include namespace DB diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index f7ba061cf3a..b36cb22498d 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 7fcb42ec378..306236534b6 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 70687ba471c..8736fb7d4e3 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 6bbd7c26bec..5d75f3b70e5 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include namespace fs = std::filesystem; diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp similarity index 72% rename from src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 24b7042e459..611fd00a023 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -1,4 +1,4 @@ -#include "AsynchronousReadIndirectBufferFromRemoteFS.h" +#include "AsynchronousBoundedReadBuffer.h" #include #include @@ -43,105 +43,77 @@ namespace ErrorCodes } -AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( +AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer( + ImplPtr impl_, IAsynchronousReader & reader_, const ReadSettings & settings_, - std::shared_ptr impl_, - std::shared_ptr async_read_counters_, - std::shared_ptr prefetches_log_) + AsyncReadCountersPtr async_read_counters_, + FilesystemReadPrefetchesLogPtr prefetches_log_) : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) + , impl(std::move(impl_)) , read_settings(settings_) , reader(reader_) - , base_priority(settings_.priority) - , impl(impl_) , prefetch_buffer(settings_.prefetch_buffer_size) - , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr - ? CurrentThread::getQueryId() : "") + , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) -#ifndef NDEBUG - , log(&Poco::Logger::get("AsynchronousBufferFromRemoteFS")) -#else - , log(&Poco::Logger::get("AsyncBuffer(" + impl->getFileName() + ")")) -#endif + , log(&Poco::Logger::get("AsynchronousBoundedReadBuffer")) , async_read_counters(async_read_counters_) , prefetches_log(prefetches_log_) { ProfileEvents::increment(ProfileEvents::RemoteFSBuffers); } -String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const +bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() { - return impl->getFileName(); -} - - -String AsynchronousReadIndirectBufferFromRemoteFS::getInfoForLog() -{ - return impl->getInfoForLog(); -} - -size_t AsynchronousReadIndirectBufferFromRemoteFS::getFileSize() -{ - return impl->getFileSize(); -} - -bool AsynchronousReadIndirectBufferFromRemoteFS::hasPendingDataToRead() -{ - /** - * Note: read_until_position here can be std::nullopt only for non-MergeTree tables. - * For mergeTree tables it must be guaranteed that setReadUntilPosition() or - * setReadUntilEnd() is called before any read or prefetch. - * setReadUntilEnd() always sets read_until_position to file size. - * setReadUntilPosition(pos) always has pos > 0, because if - * right_offset_in_compressed_file is 0, then setReadUntilEnd() is used. - */ if (read_until_position) { - /// Everything is already read. - if (file_offset_of_buffer_end == *read_until_position) + if (file_offset_of_buffer_end == *read_until_position) /// Everything is already read. return false; if (file_offset_of_buffer_end > *read_until_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {}, info: {})", - file_offset_of_buffer_end, *read_until_position, impl->getInfoForLog()); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read beyond last offset ({} > {}, info: {})", + file_offset_of_buffer_end, *read_until_position, impl->getInfoForLog()); + } } return true; } - -std::future AsynchronousReadIndirectBufferFromRemoteFS::asyncReadInto(char * data, size_t size, int64_t priority) +std::future +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, int64_t priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, async_read_counters); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = read_settings.priority + priority; request.ignore = bytes_to_ignore; return reader.submit(request); } - -void AsynchronousReadIndirectBufferFromRemoteFS::prefetch(int64_t priority) +void AsynchronousBoundedReadBuffer::prefetch(int64_t priority) { if (prefetch_future.valid()) return; - /// Check boundary, which was set in readUntilPosition(). if (!hasPendingDataToRead()) return; - last_prefetch_info.submit_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + last_prefetch_info.submit_time = std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); last_prefetch_info.priority = priority; - /// Prefetch even in case hasPendingData() == true. - chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); + chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size + || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); prefetch_future = asyncReadInto(prefetch_buffer.data(), prefetch_buffer.size(), priority); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } -void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) +void AsynchronousBoundedReadBuffer::setReadUntilPosition(size_t position) { if (!read_until_position || position != *read_until_position) { @@ -157,21 +129,16 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t pos } } - -void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilEnd() +void AsynchronousBoundedReadBuffer::appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch) { - setReadUntilPosition(impl->getFileSize()); -} - - -void AsynchronousReadIndirectBufferFromRemoteFS::appendToPrefetchLog(FilesystemPrefetchState state, int64_t size, const std::unique_ptr & execution_watch) -{ - const auto & object = impl->getCurrentObject(); FilesystemReadPrefetchesLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), .query_id = query_id, - .path = object.local_path, + .path = impl->getFileName(), .offset = file_offset_of_buffer_end, .size = size, .prefetch_submit_time = last_prefetch_info.submit_time, @@ -187,7 +154,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::appendToPrefetchLog(FilesystemP } -bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() +bool AsynchronousBoundedReadBuffer::nextImpl() { if (!hasPendingDataToRead()) return false; @@ -245,14 +212,14 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() /// In case of multiple files for the same file in clickhouse (i.e. log family) /// file_offset_of_buffer_end will not match getImplementationBufferOffset() /// so we use [impl->getImplementationBufferOffset(), impl->getFileSize()] - chassert(file_offset_of_buffer_end >= impl->getImplementationBufferOffset()); + chassert(file_offset_of_buffer_end >= impl->getFileOffsetOfBufferEnd()); chassert(file_offset_of_buffer_end <= impl->getFileSize()); return bytes_read; } -off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) +off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) { ProfileEvents::increment(ProfileEvents::RemoteFSSeeks); @@ -268,7 +235,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) } else { - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "ReadBufferFromFileDescriptor::seek expects SEEK_SET or SEEK_CUR as whence"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected SEEK_SET or SEEK_CUR as whence"); } /// Position is unchanged. @@ -322,9 +289,8 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) if (read_until_position && new_pos > *read_until_position) { ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); - impl->reset(); - file_offset_of_buffer_end = new_pos = *read_until_position; /// read_until_position is a non-included boundary. + impl->seek(file_offset_of_buffer_end, SEEK_SET); return new_pos; } @@ -332,8 +298,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (impl->initialized() - && read_until_position && new_pos < *read_until_position + if (read_until_position && new_pos < *read_until_position && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { @@ -342,31 +307,21 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) } else { - if (impl->initialized()) - { - ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); - impl->reset(); - } + ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); file_offset_of_buffer_end = new_pos; + impl->seek(file_offset_of_buffer_end, SEEK_SET); } return new_pos; } -off_t AsynchronousReadIndirectBufferFromRemoteFS::getPosition() -{ - return file_offset_of_buffer_end - available() + bytes_to_ignore; -} - - -void AsynchronousReadIndirectBufferFromRemoteFS::finalize() +void AsynchronousBoundedReadBuffer::finalize() { resetPrefetch(FilesystemPrefetchState::UNNEEDED); } - -AsynchronousReadIndirectBufferFromRemoteFS::~AsynchronousReadIndirectBufferFromRemoteFS() +AsynchronousBoundedReadBuffer::~AsynchronousBoundedReadBuffer() { try { @@ -378,7 +333,7 @@ AsynchronousReadIndirectBufferFromRemoteFS::~AsynchronousReadIndirectBufferFromR } } -void AsynchronousReadIndirectBufferFromRemoteFS::resetPrefetch(FilesystemPrefetchState state) +void AsynchronousBoundedReadBuffer::resetPrefetch(FilesystemPrefetchState state) { if (!prefetch_future.valid()) return; diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h new file mode 100644 index 00000000000..45256cdfac2 --- /dev/null +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -0,0 +1,96 @@ +#pragma once + +#include "config.h" +#include +#include +#include +#include +#include + +namespace Poco { class Logger; } + +namespace DB +{ + +struct AsyncReadCounters; +using AsyncReadCountersPtr = std::shared_ptr; +class ReadBufferFromRemoteFSGather; + +class AsynchronousBoundedReadBuffer : public ReadBufferFromFileBase +{ +public: + using Impl = ReadBufferFromFileBase; + using ImplPtr = std::unique_ptr; + + explicit AsynchronousBoundedReadBuffer( + ImplPtr impl_, + IAsynchronousReader & reader_, + const ReadSettings & settings_, + AsyncReadCountersPtr async_read_counters_ = nullptr, + FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr); + + ~AsynchronousBoundedReadBuffer() override; + + String getFileName() const override { return impl->getFileName(); } + + size_t getFileSize() override { return impl->getFileSize(); } + + String getInfoForLog() override { return impl->getInfoForLog(); } + + off_t seek(off_t offset_, int whence) override; + + void prefetch(int64_t priority) override; + + void setReadUntilPosition(size_t position) override; /// [..., position). + + void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); } + + off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } + +private: + const ImplPtr impl; + const ReadSettings read_settings; + IAsynchronousReader & reader; + + size_t file_offset_of_buffer_end = 0; + std::optional read_until_position; + /// If nonzero then working_buffer is empty. + /// If a prefetch is in flight, the prefetch task has been instructed to ignore this many bytes. + size_t bytes_to_ignore = 0; + + Memory<> prefetch_buffer; + std::future prefetch_future; + + const std::string query_id; + const std::string current_reader_id; + + Poco::Logger * log; + + AsyncReadCountersPtr async_read_counters; + FilesystemReadPrefetchesLogPtr prefetches_log; + + struct LastPrefetchInfo + { + UInt64 submit_time = 0; + size_t priority = 0; + }; + LastPrefetchInfo last_prefetch_info; + + bool nextImpl() override; + + void finalize(); + + bool hasPendingDataToRead(); + + void appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch); + + std::future asyncReadInto(char * data, size_t size, int64_t priority); + + void resetPrefetch(FilesystemPrefetchState state); + +}; + +} diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h deleted file mode 100644 index e8fb3fe248b..00000000000 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ /dev/null @@ -1,111 +0,0 @@ -#pragma once - -#include "config.h" -#include -#include -#include -#include -#include - -namespace Poco { class Logger; } - -namespace DB -{ - -struct AsyncReadCounters; -class ReadBufferFromRemoteFSGather; - -/** - * Reads data from S3/HDFS/Web using stored paths in metadata. -* This class is an asynchronous version of ReadIndirectBufferFromRemoteFS. -* -* Buffers chain for diskS3: -* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> -* -> ReadBufferFromS3 -> ReadBufferFromIStream. -* -* Buffers chain for diskWeb: -* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> -* -> ReadIndirectBufferFromWebServer -> ReadBufferFromHTTP -> ReadBufferFromIStream. -* -* We pass either `memory` or `prefetch_buffer` through all this chain and return it back. -*/ -class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase -{ -public: - explicit AsynchronousReadIndirectBufferFromRemoteFS( - IAsynchronousReader & reader_, const ReadSettings & settings_, - std::shared_ptr impl_, - std::shared_ptr async_read_counters_, - std::shared_ptr prefetches_log_); - - ~AsynchronousReadIndirectBufferFromRemoteFS() override; - - off_t seek(off_t offset_, int whence) override; - - off_t getPosition() override; - - String getFileName() const override; - - void prefetch(int64_t priority) override; - - void setReadUntilPosition(size_t position) override; /// [..., position). - - void setReadUntilEnd() override; - - String getInfoForLog() override; - - size_t getFileSize() override; - - bool isIntegratedWithFilesystemCache() const override { return true; } - -private: - bool nextImpl() override; - - void finalize(); - - bool hasPendingDataToRead(); - - void appendToPrefetchLog(FilesystemPrefetchState state, int64_t size, const std::unique_ptr & execution_watch); - - std::future asyncReadInto(char * data, size_t size, int64_t priority); - - void resetPrefetch(FilesystemPrefetchState state); - - ReadSettings read_settings; - - IAsynchronousReader & reader; - - int64_t base_priority; - - std::shared_ptr impl; - - std::future prefetch_future; - - size_t file_offset_of_buffer_end = 0; - - Memory<> prefetch_buffer; - - std::string query_id; - - std::string current_reader_id; - - /// If nonzero then working_buffer is empty. - /// If a prefetch is in flight, the prefetch task has been instructed to ignore this many bytes. - size_t bytes_to_ignore = 0; - - std::optional read_until_position; - - Poco::Logger * log; - - std::shared_ptr async_read_counters; - std::shared_ptr prefetches_log; - - struct LastPrefetchInfo - { - UInt64 submit_time = 0; - size_t priority = 0; - }; - LastPrefetchInfo last_prefetch_info; -}; - -} diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 68b5a9c9d96..8db93e3ed6e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -12,22 +12,24 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_SEEK_THROUGH_FILE; +} ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_, std::shared_ptr cache_log_) - : ReadBuffer(nullptr, 0) - , read_buffer_creator(std::move(read_buffer_creator_)) - , blobs_to_read(blobs_to_read_) + : ReadBufferFromFileBase(0, nullptr, 0) , settings(settings_) + , blobs_to_read(blobs_to_read_) + , read_buffer_creator(std::move(read_buffer_creator_)) + , cache_log(settings.enable_filesystem_cache_log ? cache_log_ : nullptr) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) { - if (cache_log_ && settings.enable_filesystem_cache_log) - cache_log = cache_log_; - if (!blobs_to_read.empty()) current_object = blobs_to_read.front(); @@ -38,9 +40,9 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) { - if (current_buf != nullptr && !with_cache) + if (current_buf && !with_cache) { - appendFilesystemCacheLog(); + appendUncachedReadInfo(); } current_object = object; @@ -70,7 +72,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c return current_read_buffer_creator(); } -void ReadBufferFromRemoteFSGather::appendFilesystemCacheLog() +void ReadBufferFromRemoteFSGather::appendUncachedReadInfo() { if (!cache_log || current_object.remote_path.empty()) return; @@ -218,44 +220,23 @@ bool ReadBufferFromRemoteFSGather::readImpl() return result; } -size_t ReadBufferFromRemoteFSGather::getFileOffsetOfBufferEnd() const -{ - return file_offset_of_buffer_end; -} - void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) { - if (position != read_until_position) - { - read_until_position = position; - reset(); - } -} + if (position == read_until_position) + return; -void ReadBufferFromRemoteFSGather::reset() -{ + read_until_position = position; current_buf.reset(); } -String ReadBufferFromRemoteFSGather::getFileName() const +off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) { - return current_object.remote_path; -} + if (whence != SEEK_SET) + throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only seeking with SEEK_SET is allowed"); -size_t ReadBufferFromRemoteFSGather::getFileSize() const -{ - size_t size = 0; - for (const auto & object : blobs_to_read) - size += object.bytes_size; - return size; -} - -String ReadBufferFromRemoteFSGather::getInfoForLog() -{ - if (!current_buf) - return ""; - - return current_buf->getInfoForLog(); + file_offset_of_buffer_end = offset; + current_buf.reset(); + return file_offset_of_buffer_end; } size_t ReadBufferFromRemoteFSGather::getImplementationBufferOffset() const @@ -269,7 +250,7 @@ size_t ReadBufferFromRemoteFSGather::getImplementationBufferOffset() const ReadBufferFromRemoteFSGather::~ReadBufferFromRemoteFSGather() { if (!with_cache) - appendFilesystemCacheLog(); + appendUncachedReadInfo(); } } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 8c55f747e5b..08e5b97d018 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -10,12 +10,13 @@ namespace Poco { class Logger; } namespace DB { +class FilesystemCacheLog; /** * Remote disk might need to split one clickhouse file into multiple files in remote fs. * This class works like a proxy to allow transition from one file into multiple. */ -class ReadBufferFromRemoteFSGather final : public ReadBuffer +class ReadBufferFromRemoteFSGather final : public ReadBufferFromFileBase { friend class ReadIndirectBufferFromRemoteFS; @@ -30,25 +31,25 @@ public: ~ReadBufferFromRemoteFSGather() override; - String getFileName() const; + String getFileName() const override { return current_object.remote_path; } - void reset(); + String getInfoForLog() override { return current_buf ? current_buf->getInfoForLog() : ""; } void setReadUntilPosition(size_t position) override; IAsynchronousReader::Result readInto(char * data, size_t size, size_t offset, size_t ignore) override; - size_t getFileSize() const; + size_t getFileSize() override { return getTotalSize(blobs_to_read); } - size_t getFileOffsetOfBufferEnd() const; + size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } bool initialized() const { return current_buf != nullptr; } - String getInfoForLog(); - size_t getImplementationBufferOffset() const; - const StoredObject & getCurrentObject() const { return current_object; } + off_t seek(off_t offset, int whence) override; + + off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } private: SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); @@ -61,40 +62,25 @@ private: bool moveToNextBuffer(); - void appendFilesystemCacheLog(); - - ReadBufferCreator read_buffer_creator; - - StoredObjects blobs_to_read; - - ReadSettings settings; - - size_t read_until_position = 0; - - StoredObject current_object; + void appendUncachedReadInfo(); + const ReadSettings settings; + const StoredObjects blobs_to_read; + const ReadBufferCreator read_buffer_creator; + const std::shared_ptr cache_log; + const String query_id; bool with_cache; - String query_id; - - Poco::Logger * log; - - SeekableReadBufferPtr current_buf; - - size_t current_buf_idx = 0; - + size_t read_until_position = 0; size_t file_offset_of_buffer_end = 0; - - /** - * File: |___________________| - * Buffer: |~~~~~~~| - * file_offset_of_buffer_end: ^ - */ size_t bytes_to_ignore = 0; - size_t total_bytes_read_from_current_file = 0; - std::shared_ptr cache_log; + StoredObject current_object; + size_t current_buf_idx = 0; + SeekableReadBufferPtr current_buf; + + Poco::Logger * log; }; } diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index 8a33a6ce9a1..04521011599 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -82,8 +82,8 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET or SEEK_CUR modes are allowed."); - impl->reset(); resetWorkingBuffer(); + impl->seek(file_offset_of_buffer_end, SEEK_SET); file_offset_of_buffer_end = impl->file_offset_of_buffer_end; return impl->file_offset_of_buffer_end; diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h index 0424c2e56d3..19647b1fa39 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h @@ -31,8 +31,6 @@ public: void setReadUntilEnd() override; - bool isIntegratedWithFilesystemCache() const override { return true; } - size_t getFileSize() override; private: diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..997002a1e4f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -112,8 +113,8 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, disk_read_settings, std::move(reader_impl), + return std::make_unique( + std::move(reader_impl), reader, disk_read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 648016fb732..9616fd84770 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h index 19011a04722..cb8d9b8a5af 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..122414e3082 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 52e8b1a465d..9d3e09bd4a7 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..08969943353 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -12,12 +12,14 @@ #include #include #include +#include -#include #include #include #include #include +#include +#include namespace DB diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 8cf0b27a517..f3da39d6866 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -64,12 +65,12 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL global_context->getFilesystemCacheLog()); /// We use `remove_fs_method` (not `local_fs_method`) because we are about to use - /// AsynchronousReadIndirectBufferFromRemoteFS which works by the remote_fs_* settings. + /// AsynchronousBoundedReadBuffer which works by the remote_fs_* settings. if (modified_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, modified_settings, std::move(impl), + return std::make_unique( + std::move(impl), reader, modified_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..89159fc2d5a 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include @@ -128,8 +128,8 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, disk_read_settings, std::move(s3_impl), + return std::make_unique( + std::move(s3_impl), reader, disk_read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/StoredObject.cpp b/src/Disks/ObjectStorages/StoredObject.cpp new file mode 100644 index 00000000000..6a363c64107 --- /dev/null +++ b/src/Disks/ObjectStorages/StoredObject.cpp @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +size_t getTotalSize(const StoredObjects & objects) +{ + size_t size = 0; + for (const auto & object : objects) + size += object.bytes_size; + return size; +} + +} diff --git a/src/Disks/ObjectStorages/StoredObject.h b/src/Disks/ObjectStorages/StoredObject.h index 94c9fd0946d..8afbb116a83 100644 --- a/src/Disks/ObjectStorages/StoredObject.h +++ b/src/Disks/ObjectStorages/StoredObject.h @@ -29,4 +29,6 @@ struct StoredObject using StoredObjects = std::vector; +size_t getTotalSize(const StoredObjects & objects); + } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index c57b75f4038..4aa0ac5708c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include #include @@ -190,8 +191,8 @@ std::unique_ptr WebObjectStorage::readObject( /// NOLINT if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, read_settings, std::move(web_impl), + return std::make_unique( + std::move(web_impl), reader, read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/IO/AsyncReadCounters.h b/src/IO/AsyncReadCounters.h index 1d941b4fc47..1f84b2a214b 100644 --- a/src/IO/AsyncReadCounters.h +++ b/src/IO/AsyncReadCounters.h @@ -27,5 +27,6 @@ struct AsyncReadCounters void dumpToMapColumn(IColumn * column) const; }; +using AsyncReadCountersPtr = std::shared_ptr; } diff --git a/src/IO/ReadBufferFromFileDecorator.h b/src/IO/ReadBufferFromFileDecorator.h index 1d035e8d74b..6e62c7f741b 100644 --- a/src/IO/ReadBufferFromFileDecorator.h +++ b/src/IO/ReadBufferFromFileDecorator.h @@ -27,8 +27,6 @@ public: ReadBuffer & getWrappedReadBuffer() { return *impl; } - bool isIntegratedWithFilesystemCache() const override { return impl->isIntegratedWithFilesystemCache(); } - size_t getFileSize() override; protected: diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index b055aa57975..736ab5bbc71 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -49,8 +49,6 @@ public: /// If true, setReadUntilPosition() guarantees that eof will be reported at the given position. virtual bool supportsRightBoundedReads() const { return false; } - virtual bool isIntegratedWithFilesystemCache() const { return false; } - /// Returns true if seek() actually works, false if seek() will always throw (or make subsequent /// nextImpl() calls throw). /// diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index a7672c49d91..7052cf2769d 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -45,4 +45,6 @@ public: using SystemLog::SystemLog; }; +using FilesystemReadPrefetchesLogPtr = std::shared_ptr; + } diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 503b4aac51d..09a04f13fc7 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a4d9dc9f2e3..b8df218ad8a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -31,7 +31,7 @@ #include #include -#include +#include #include #include @@ -676,8 +676,8 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - pool_reader, modified_settings, std::move(s3_impl), + auto async_reader = std::make_unique( + std::move(s3_impl), pool_reader, modified_settings, context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); async_reader->setReadUntilEnd(); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 32721a0020b..00b5dbfc5e3 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include From d5cfcdfae10c3706ad2e33d0e3a1f52e6c59ece0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:09:11 +0000 Subject: [PATCH 21/59] String terminator: \n --> \0 --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index afdc01b717d..dd30e548f30 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -70,7 +70,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; @@ -107,7 +107,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; From d76498dca06962873d9fa654d9cda55937135a82 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:19:08 +0000 Subject: [PATCH 22/59] reserve() --> resize() --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index dd30e548f30..70e4d3e6794 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -88,7 +88,7 @@ public: return false; res_offsets.resize(col_times->size()); - res_chars.reserve(col_times->size() * 10); /// heuristic + res_chars.resize(col_times->size() * 10); /// heuristic const PaddedPODArray & times_data = col_times->getData(); @@ -102,7 +102,7 @@ public: times = 0; if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.reserve(2 * res_chars.capacity()); + res_chars.resize(2 * res_chars.capacity()); memset(res_chars.begin() + pos, space, times); pos += times; From 786bbee62cec5ec55750eaace43bebae683d6197 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 20:03:29 +0000 Subject: [PATCH 23/59] Extend tests --- .../02752_space_function.reference | 69 ++++++++++++++----- .../0_stateless/02752_space_function.sql | 47 ++++++++----- 2 files changed, 82 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference index b0adccb415b..d265a843ba9 100644 --- a/tests/queries/0_stateless/02752_space_function.reference +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -1,21 +1,56 @@ -const times, uint - - - - -const times, int - - - - -negative const times - - - - +const, uint + 3 + 3 + 3 + 3 +const, int + 3 + 3 + 3 + 3 +const, int, negative + 0 + 0 + 0 + 0 negative tests +null \N -non-const times, uint +const, uint, multiple + + + + + + + + + + + + + + + + +const int, multiple + + + + + + + + + + + + + + + + +non-const, uint 3 2 1 @@ -32,7 +67,7 @@ non-const times, uint 20 5 7 -non-const times, int +non-const, int 3 2 1 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index 8f3605e6e6f..d621af13096 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -1,24 +1,26 @@ -SELECT 'const times, uint'; -SELECT space(3::UInt8); -SELECT space(3::UInt16); -SELECT space(3::UInt32); -SELECT space(3::UInt64); -SELECT 'const times, int'; -SELECT space(3::Int8); -SELECT space(3::Int16); -SELECT space(3::Int32); -SELECT space(3::Int64); +SELECT 'const, uint'; +SELECT space(3::UInt8), length(space(3::UInt8)); +SELECT space(3::UInt16), length(space(3::UInt16)); +SELECT space(3::UInt32), length(space(3::UInt32)); +SELECT space(3::UInt64), length(space(3::UInt64)); +SELECT 'const, int'; +SELECT space(3::Int8), length(space(3::Int8)); +SELECT space(3::Int16), length(space(3::Int16)); +SELECT space(3::Int32), length(space(3::Int32)); +SELECT space(3::Int64), length(space(3::Int64)); -SELECT 'negative const times'; -SELECT space(-3::Int8); -SELECT space(-3::Int16); -SELECT space(-3::Int32); -SELECT space(-3::Int64); +SELECT 'const, int, negative'; +SELECT space(-3::Int8), length(space(-3::Int8)); +SELECT space(-3::Int16), length(space(-3::Int16)); +SELECT space(-3::Int32), length(space(-3::Int32)); +SELECT space(-3::Int64), length(space(-3::Int64)); SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'null'; SELECT space(NULL); DROP TABLE IF EXISTS defaults; @@ -36,12 +38,23 @@ CREATE TABLE defaults INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); -SELECT 'non-const times, uint'; +SELECT 'const, uint, multiple'; +SELECT space(30::UInt8) FROM defaults; +SELECT space(30::UInt16) FROM defaults; +SELECT space(30::UInt32) FROM defaults; +SELECT space(30::UInt64) FROM defaults; +SELECT 'const int, multiple'; +SELECT space(30::Int8) FROM defaults; +SELECT space(30::Int16) FROM defaults; +SELECT space(30::Int32) FROM defaults; +SELECT space(30::Int64) FROM defaults; + +SELECT 'non-const, uint'; SELECT space(u8), length(space(u8)) FROM defaults; SELECT space(u16), length(space(u16)) FROM defaults; SELECT space(u32), length(space(u32)) from defaults; SELECT space(u64), length(space(u64)) FROM defaults; -SELECT 'non-const times, int'; +SELECT 'non-const, int'; SELECT space(i8), length(space(i8)) FROM defaults; SELECT space(i16), length(space(i16)) FROM defaults; SELECT space(i32), length(space(i32)) from defaults; From ce6054590fa3b294783b73edd4f5d589b32fc4a7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 22 May 2023 22:49:09 +0200 Subject: [PATCH 24/59] Fix bad merge --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index f776de2f7bc..05c0c8f3961 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From 3a29f275e00f3a0199b9e64ca66833e61e2545c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 21:50:12 +0000 Subject: [PATCH 25/59] Fix: do not generate suffix on new chunk if didn't reach current range end --- src/Processors/Transforms/FillingTransform.cpp | 2 +- ...02730_with_fill_by_sorting_prefix.reference | 18 +++++++++--------- .../02730_with_fill_by_sorting_prefix.sql | 16 ++++++++-------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 052a5a1c183..4d28c079dd0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -786,7 +786,7 @@ void FillingTransform::transform(Chunk & chunk) }); /// generate suffix for the previous range - if (!last_range_sort_prefix.empty()) + if (!last_range_sort_prefix.empty() && new_sort_prefix) generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); transformRange( diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 3e2f939af28..cf65b270fc9 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -25,10 +25,11 @@ select * from ts order by sensor_id, timestamp with fill step 1; drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; --- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); -insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; 1 10 1 1 11 0 1 12 1 @@ -41,11 +42,10 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; --- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1); -insert into ts VALUES (3, 5, 1), (3, 7, 1); -insert into ts VALUES (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; 1 10 1 1 11 0 1 12 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 01fc125672a..42f2b93ec73 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -14,14 +14,6 @@ insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), ( -- FillingTransform: 6 rows will be processed in 1 chunks select * from ts order by sensor_id, timestamp with fill step 1; -drop table if exists ts; -create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); -system stop merges ts; --- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); -insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; - drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; @@ -31,6 +23,14 @@ insert into ts VALUES (3, 5, 1), (3, 7, 1); insert into ts VALUES (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; + select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; From 1d51488d3d98c505cd5c19918850bb75a0449345 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 22:14:16 +0000 Subject: [PATCH 26/59] Tests: w/o TO, w/o FROM --- ...2730_with_fill_by_sorting_prefix.reference | 51 +++++++++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 8 +++ 2 files changed, 59 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index cf65b270fc9..7f1bcc838e7 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -84,5 +84,56 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 3 7 1 5 1 1 5 3 1 +-- without TO +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 11 9999 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +5 1 1 +5 3 1 +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +0 6 9999 +0 7 9999 +0 8 9999 +0 9 9999 +1 10 1 +0 11 9999 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 +-- -- without FROM +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); +1 10 1 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +5 1 1 +5 2 9999 +5 3 1 +5 4 9999 +5 5 9999 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +1 10 1 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 42f2b93ec73..dd0f06e5c8c 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -34,5 +34,13 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- without TO +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + +-- without FROM +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From e1b00516c937e566a9149ddf48d2df7441a6a0fb Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 05:13:19 +0000 Subject: [PATCH 27/59] Update test reference --- .../0_stateless/02730_with_fill_by_sorting_prefix.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 7f1bcc838e7..5ec4245b478 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -110,7 +110,7 @@ select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpola 3 7 1 5 1 1 5 3 1 --- -- without FROM +-- without FROM select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); 1 10 1 1 12 1 From d9a7227cf414caad0b45f921e1805f7eeaed3520 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:49:19 +0000 Subject: [PATCH 28/59] Fix style check --- src/Functions/repeat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 88eaebf4f03..93f569d3eea 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_LARGE_STRING_SIZE; } From b106757a3cb7417b32438706b290fbd1013c8fd7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:51:59 +0000 Subject: [PATCH 29/59] Fix 02415_all_new_functions_must_be_documented --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 79a6ad1fa2d..7ab26982402 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -655,6 +655,7 @@ sleep sleepEachRow snowflakeToDateTime snowflakeToDateTime64 +space splitByChar splitByNonAlpha splitByRegexp From d26f9e4022ae119d03b612a263f110f637cce93d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 10:24:06 +0000 Subject: [PATCH 30/59] Test with DESC order in sorting prefix --- ...2730_with_fill_by_sorting_prefix.reference | 88 +++++++++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 14 +++ 2 files changed, 102 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 5ec4245b478..260f08027f1 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -55,6 +55,8 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo 5 1 1 5 2 0 5 3 1 +-- FROM and TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); 1 6 9999 1 7 9999 @@ -84,7 +86,37 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +5 1 1 +5 3 1 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +5 3 1 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +1 12 1 -- without TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); 1 6 9999 1 7 9999 @@ -110,7 +142,33 @@ select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpola 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999); +5 1 1 +5 3 1 +3 5 1 +3 6 9999 +3 7 1 +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 11 9999 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +5 3 1 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +0 11 9999 +1 12 1 -- without FROM +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); 1 10 1 1 12 1 @@ -135,5 +193,35 @@ select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolat 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999); +5 1 1 +5 2 9999 +5 3 1 +5 4 9999 +5 5 9999 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +1 10 1 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +0 2 9999 +5 3 1 +0 4 9999 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +1 12 1 -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index dd0f06e5c8c..e2f1ce29d5c 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -31,16 +31,30 @@ insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +-- FROM and TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + -- without TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -- without FROM +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From 241e75197ee64ea1c7578d35f5aa3cd3cceda968 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 May 2023 13:31:50 +0200 Subject: [PATCH 31/59] Fix --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 24 +++++++++++-------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 3 ++- .../IO/ReadIndirectBufferFromRemoteFS.cpp | 2 +- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8db93e3ed6e..12fbbbcf747 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -46,7 +47,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c } current_object = object; - total_bytes_read_from_current_file = 0; const auto & object_path = object.remote_path; size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size; @@ -84,7 +84,7 @@ void ReadBufferFromRemoteFSGather::appendUncachedReadInfo() .source_file_path = current_object.remote_path, .file_segment_range = { 0, current_object.bytes_size }, .cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE, - .file_segment_size = total_bytes_read_from_current_file, + .file_segment_size = current_object.bytes_size, .read_from_cache_attempted = false, }; cache_log->add(elem); @@ -176,7 +176,7 @@ bool ReadBufferFromRemoteFSGather::moveToNextBuffer() bool ReadBufferFromRemoteFSGather::readImpl() { - swap(*current_buf); + SwapHelper swap(*this, *current_buf); bool result = false; @@ -187,7 +187,6 @@ bool ReadBufferFromRemoteFSGather::readImpl() */ if (bytes_to_ignore) { - total_bytes_read_from_current_file += bytes_to_ignore; current_buf->ignore(bytes_to_ignore); result = current_buf->hasPendingData(); file_offset_of_buffer_end += bytes_to_ignore; @@ -207,14 +206,11 @@ bool ReadBufferFromRemoteFSGather::readImpl() file_offset_of_buffer_end += current_buf->available(); } - swap(*current_buf); - /// Required for non-async reads. if (result) { - assert(available()); - nextimpl_working_buffer_offset = offset(); - total_bytes_read_from_current_file += available(); + assert(current_buf->available()); + nextimpl_working_buffer_offset = current_buf->offset(); } return result; @@ -225,8 +221,16 @@ void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) if (position == read_until_position) return; + reset(); read_until_position = position; +} + +void ReadBufferFromRemoteFSGather::reset() +{ + current_object = {}; + current_buf_idx = {}; current_buf.reset(); + bytes_to_ignore = 0; } off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) @@ -234,8 +238,8 @@ off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only seeking with SEEK_SET is allowed"); + reset(); file_offset_of_buffer_end = offset; - current_buf.reset(); return file_offset_of_buffer_end; } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 08e5b97d018..39b81d6f9ac 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -64,6 +64,8 @@ private: void appendUncachedReadInfo(); + void reset(); + const ReadSettings settings; const StoredObjects blobs_to_read; const ReadBufferCreator read_buffer_creator; @@ -74,7 +76,6 @@ private: size_t read_until_position = 0; size_t file_offset_of_buffer_end = 0; size_t bytes_to_ignore = 0; - size_t total_bytes_read_from_current_file = 0; StoredObject current_object; size_t current_buf_idx = 0; diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index 04521011599..a559b47f2cc 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -82,8 +82,8 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET or SEEK_CUR modes are allowed."); + impl->seek(impl->file_offset_of_buffer_end, SEEK_SET); resetWorkingBuffer(); - impl->seek(file_offset_of_buffer_end, SEEK_SET); file_offset_of_buffer_end = impl->file_offset_of_buffer_end; return impl->file_offset_of_buffer_end; From 285e8f4ae1f8bc3dc8a8b1d7e12bc152debc0650 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 12:16:49 +0000 Subject: [PATCH 32/59] Protect against DOS --- src/Functions/repeat.cpp | 8 +++----- src/Functions/space.cpp | 13 +++++++++++++ tests/queries/0_stateless/02752_space_function.sql | 1 + 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 93f569d3eea..c1b553ac6b3 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -24,18 +24,16 @@ struct RepeatImpl /// Safety threshold against DoS. static inline void checkRepeatTime(UInt64 repeat_time) { - static constexpr UInt64 max_repeat_times = 1000000; + static constexpr UInt64 max_repeat_times = 1'000'000; if (repeat_time > max_repeat_times) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", - std::to_string(repeat_time), std::to_string(max_repeat_times)); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } static inline void checkStringSize(UInt64 size) { static constexpr UInt64 max_string_size = 1 << 30; if (size > max_string_size) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", - size, max_string_size); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", size, max_string_size); } template diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 70e4d3e6794..9d825a8b294 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int TOO_LARGE_STRING_SIZE; } namespace @@ -25,6 +26,14 @@ class FunctionSpace : public IFunction private: static constexpr auto space = ' '; + /// Safety threshold against DoS. + static inline void checkRepeatTime(UInt64 repeat_time) + { + static constexpr UInt64 max_repeat_times = 1'000'000; + if (repeat_time > max_repeat_times) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); + } + public: static constexpr auto name = "space"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -60,6 +69,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + res_offsets.resize(col_times->size()); res_chars.resize(col_times->size() * (times + 1)); @@ -101,6 +112,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + if (res_chars.size() + times + 1 >= res_chars.capacity()) res_chars.resize(2 * res_chars.capacity()); diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index d621af13096..b12906927df 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -19,6 +19,7 @@ SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(30303030303030303030303030303030::UInt64); -- { serverError TOO_LARGE_STRING_SIZE } SELECT 'null'; SELECT space(NULL); From 8645af5809e280e89d562476657564fa74e31f7b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 14:54:22 +0000 Subject: [PATCH 33/59] Hoping to get into next release --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Processors/Transforms/FillingTransform.cpp | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9b9dc43e32e..8967c55b3e2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -729,7 +729,7 @@ class IColumn; M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \ M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function JSON_VALUE to return nullable type.", 0) \ M(Bool, function_json_value_return_type_allow_complex, false, "Allow function JSON_VALUE to return complex type, such as: struct, array, map.", 0) \ - M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause from sorting prefix. Gaps for rows with the different values in sorting prefix will be filled independently", 0) \ + M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 388cad54791..c0f10b13282 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -82,6 +82,7 @@ static std::map sett { {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4d28c079dd0..e75f83b8c80 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -14,7 +14,7 @@ namespace DB { -constexpr bool debug_logging_enabled = true; +constexpr bool debug_logging_enabled = false; template void logDebug(String key, const T & value, const char * separator = " : ") From fbeba9b5911dbaf69efbcdddf29c68d0a786ff89 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 22:55:58 +0200 Subject: [PATCH 34/59] Documentation --- .../statements/select/order-by.md | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 62feca9ecf6..f1efd6c4718 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -544,6 +544,54 @@ Result: └─────┴──────────┴───────┘ ``` +##Filling grouped by sorting prefix + +It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series. +Assume there is the following time series table +``` sql +CREATE TABLE timeseries +( + `sensor_id` UInt64, + `timestamp` DateTime64(3, 'UTC'), + `value` Float64 +) +ENGINE = Memory; + +SELECT * FROM timeseries; + +┌─sensor_id─┬───────────────timestamp─┬─value─┐ +│ 234 │ 2021-12-01 00:00:03.000 │ 3 │ +│ 432 │ 2021-12-01 00:00:01.000 │ 1 │ +│ 234 │ 2021-12-01 00:00:07.000 │ 7 │ +│ 432 │ 2021-12-01 00:00:05.000 │ 5 │ +└───────────┴─────────────────────────┴───────┘ +``` +And we'd like to fill missing values for each sensor independently with 1 second interval. +The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp` +``` +SELECT * +FROM timeseries +ORDER BY + sensor_id, + timestamp WITH FILL +INTERPOLATE ( value AS 9999 ) + +┌─sensor_id─┬───────────────timestamp─┬─value─┐ +│ 234 │ 2021-12-01 00:00:03.000 │ 3 │ +│ 234 │ 2021-12-01 00:00:04.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:05.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:06.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:07.000 │ 7 │ +│ 432 │ 2021-12-01 00:00:01.000 │ 1 │ +│ 432 │ 2021-12-01 00:00:02.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:03.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:04.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:05.000 │ 5 │ +└───────────┴─────────────────────────┴───────┘ +``` +Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable +This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default) + ## Related content - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) From da59d8a5b78b54eff73eea74fec2a2cc0ee5bdba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 May 2023 00:27:29 +0200 Subject: [PATCH 35/59] Speed-up the shellcheck with parallel xargs --- utils/check-style/shellcheck-run.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/utils/check-style/shellcheck-run.sh b/utils/check-style/shellcheck-run.sh index c0063d4b191..bdb0f681c31 100755 --- a/utils/check-style/shellcheck-run.sh +++ b/utils/check-style/shellcheck-run.sh @@ -1,9 +1,14 @@ #!/usr/bin/env bash ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New' +NPROC=$(($(nproc) + 3)) # Check sh tests with Shellcheck -(cd $ROOT_PATH/tests/queries/0_stateless/ && shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086,SC2016 *.sh ../1_stateful/*.sh) +( cd "$ROOT_PATH/tests/queries/0_stateless/" && \ + find "$ROOT_PATH/tests/queries/"{0_stateless,1_stateful} -name '*.sh' -print0 | \ + xargs -0 -P "$NPROC" -n 20 shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086,SC2016 +) # Check docker scripts with shellcheck -find "$ROOT_PATH/docker" -executable -type f -exec file -F' ' --mime-type {} \; | awk -F' ' '$2==" text/x-shellscript" {print $1}' | grep -v "entrypoint.alpine.sh" | grep -v "compare.sh"| xargs shellcheck - +find "$ROOT_PATH/docker" -executable -type f -exec file -F' ' --mime-type {} \; | \ + awk -F' ' '$2==" text/x-shellscript" {print $1}' | \ + grep -v "compare.sh" | \ + xargs -P "$NPROC" -n 20 shellcheck From 91eb3ad2bca29ec44d1d996740bca4cd026406c2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 May 2023 12:14:15 +0200 Subject: [PATCH 36/59] fix clang-tidy build --- src/Disks/IDiskTransaction.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index f0c32e04f48..935cd6b2c65 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { From 2255b0287a7c0e83d1a748f4c716096ac2673581 Mon Sep 17 00:00:00 2001 From: helifu Date: Mon, 8 May 2023 18:01:24 +0800 Subject: [PATCH 37/59] Add 'partitions' field for system.query_log --- src/Interpreters/Context.cpp | 14 ++++++++++++++ src/Interpreters/Context.h | 4 ++++ src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Interpreters/QueryLog.cpp | 3 +++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/executeQuery.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++++++++++ src/Storages/SelectQueryInfo.h | 1 + 8 files changed, 38 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5ec12d7e0ea..2fb065600fc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1386,6 +1386,20 @@ void Context::addQueryAccessInfo( query_access_info.views.emplace(view_name); } +void Context::addQueryAccessInfo(const Names & partition_names) +{ + if (isGlobalContext()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); + } + + std::lock_guard lock(query_access_info.mutex); + for (const auto & partition_name : partition_names) + { + query_access_info.partitions.emplace(partition_name); + } +} + void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const { if (isGlobalContext()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3862984bb6f..fcf035cefca 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -295,6 +295,7 @@ private: databases = rhs.databases; tables = rhs.tables; columns = rhs.columns; + partitions = rhs.partitions; projections = rhs.projections; views = rhs.views; } @@ -312,6 +313,7 @@ private: std::swap(databases, rhs.databases); std::swap(tables, rhs.tables); std::swap(columns, rhs.columns); + std::swap(partitions, rhs.partitions); std::swap(projections, rhs.projections); std::swap(views, rhs.views); } @@ -321,6 +323,7 @@ private: std::set databases{}; std::set tables{}; std::set columns{}; + std::set partitions{}; std::set projections{}; std::set views{}; }; @@ -629,6 +632,7 @@ public: const Names & column_names, const String & projection_name = {}, const String & view_name = {}); + void addQueryAccessInfo(const Names & partition_names); /// Supported factories for records in query_log diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d68f9c8e6a6..c410ec998f5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -385,6 +385,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; + query_info.is_internal = options.is_internal; initSettings(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 4746954edf2..ec0315c2f95 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -70,6 +70,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"databases", array_low_cardinality_string}, {"tables", array_low_cardinality_string}, {"columns", array_low_cardinality_string}, + {"partitions", array_low_cardinality_string}, {"projections", array_low_cardinality_string}, {"views", array_low_cardinality_string}, {"exception_code", std::make_shared()}, @@ -176,6 +177,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_databases = typeid_cast(*columns[i++]); auto & column_tables = typeid_cast(*columns[i++]); auto & column_columns = typeid_cast(*columns[i++]); + auto & column_partitions = typeid_cast(*columns[i++]); auto & column_projections = typeid_cast(*columns[i++]); auto & column_views = typeid_cast(*columns[i++]); @@ -194,6 +196,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(query_databases, column_databases); fill_column(query_tables, column_tables); fill_column(query_columns, column_columns); + fill_column(query_partitions, column_partitions); fill_column(query_projections, column_projections); fill_column(query_views, column_views); } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 44780f530e0..570d1297239 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -65,6 +65,7 @@ struct QueryLogElement std::set query_databases; std::set query_tables; std::set query_columns; + std::set query_partitions; std::set query_projections; std::set query_views; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9a2750f399c..7ee28fce665 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -837,6 +837,7 @@ static std::tuple executeQueryImpl( elem.query_databases = info.databases; elem.query_tables = info.tables; elem.query_columns = info.columns; + elem.query_partitions = info.partitions; elem.query_projections = info.projections; elem.query_views = info.views; } @@ -901,6 +902,7 @@ static std::tuple executeQueryImpl( element.query_databases.insert(access_info.databases.begin(), access_info.databases.end()); element.query_tables.insert(access_info.tables.begin(), access_info.tables.end()); element.query_columns.insert(access_info.columns.begin(), access_info.columns.end()); + element.query_partitions.insert(access_info.partitions.begin(), access_info.partitions.end()); element.query_projections.insert(access_info.projections.begin(), access_info.projections.end()); element.query_views.insert(access_info.views.begin(), access_info.views.end()); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1c5b625656c..6180a01dec6 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1618,6 +1618,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons result.selected_marks, result.selected_ranges); + // Adding partition info to QueryAccessInfo. + if (context->hasQueryContext() && !query_info.is_internal) + { + Names partition_names; + for (const auto & part : result.parts_with_ranges) + { + partition_names.emplace_back( + fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); + } + context->getQueryContext()->addQueryAccessInfo(partition_names); + } + ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); ProfileEvents::increment(ProfileEvents::SelectedRanges, result.selected_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, result.selected_marks); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fb895d04b8f..b3dfd44b2ad 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -251,6 +251,7 @@ struct SelectQueryInfo bool is_projection_query = false; bool merge_tree_empty_result = false; bool settings_limit_offset_done = false; + bool is_internal = false; Block minmax_count_projection_block; MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr; From e138289fbf12ba0cbc944773f74e3a943c7c1961 Mon Sep 17 00:00:00 2001 From: helifu Date: Tue, 16 May 2023 20:09:49 +0800 Subject: [PATCH 38/59] Test the 'partitions' field of system.query_log --- .../02751_query_log_test_partitions.reference | 2 ++ .../02751_query_log_test_partitions.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02751_query_log_test_partitions.reference create mode 100644 tests/queries/0_stateless/02751_query_log_test_partitions.sql diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.reference b/tests/queries/0_stateless/02751_query_log_test_partitions.reference new file mode 100644 index 00000000000..8419e7f9786 --- /dev/null +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.reference @@ -0,0 +1,2 @@ +3 3 +02751_query_log_test_partitions.3 \ No newline at end of file diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.sql b/tests/queries/0_stateless/02751_query_log_test_partitions.sql new file mode 100644 index 00000000000..f30b0ee1492 --- /dev/null +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.sql @@ -0,0 +1,20 @@ +set log_queries=1; +set log_queries_min_type='QUERY_FINISH'; + +DROP TABLE IF EXISTS 02751_query_log_test_partitions; +CREATE TABLE 02751_query_log_test_partitions (a Int64, b Int64) ENGINE = MergeTree PARTITION BY a ORDER BY b; + +SYSTEM STOP MERGES 02751_query_log_test_partitions; +INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(5); + +SELECT * FROM 02751_query_log_test_partitions WHERE a = 3; +SYSTEM FLUSH LOGS; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', partitions[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and + query = 'SELECT * FROM 02751_query_log_test_partitions WHERE a = 3;' From 880745453d29e4b5f8d641ccc614b8f8aaf79575 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 10:09:04 +0800 Subject: [PATCH 39/59] small update for test case --- tests/queries/0_stateless/02751_query_log_test_partitions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.sql b/tests/queries/0_stateless/02751_query_log_test_partitions.sql index f30b0ee1492..be047d1a46e 100644 --- a/tests/queries/0_stateless/02751_query_log_test_partitions.sql +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.sql @@ -4,10 +4,10 @@ set log_queries_min_type='QUERY_FINISH'; DROP TABLE IF EXISTS 02751_query_log_test_partitions; CREATE TABLE 02751_query_log_test_partitions (a Int64, b Int64) ENGINE = MergeTree PARTITION BY a ORDER BY b; -SYSTEM STOP MERGES 02751_query_log_test_partitions; -INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(5); +INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(10); SELECT * FROM 02751_query_log_test_partitions WHERE a = 3; + SYSTEM FLUSH LOGS; SELECT From 4724745b4cbcc6022f796e9b3035c79581d21f76 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 16:15:22 +0800 Subject: [PATCH 40/59] Add the 'partitions' field description in docs --- docs/en/operations/system-tables/query_log.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 1bcecfeb161..71e1452cef1 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -59,9 +59,10 @@ Columns: - `query_kind` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Type of the query. - `databases` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the databases present in the query. - `tables` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the tables present in the query. -- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the (materialized or live) views present in the query. - `columns` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the columns present in the query. +- `partitions` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the partitions present in the query. - `projections` ([String](../../sql-reference/data-types/string.md)) — Names of the projections used during the query execution. +- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the (materialized or live) views present in the query. - `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code of an exception. - `exception` ([String](../../sql-reference/data-types/string.md)) — Exception message. - `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [Stack trace](https://en.wikipedia.org/wiki/Stack_trace). An empty string, if the query was completed successfully. From 62208feaf3149e3af94ec25f6887b9e58d524f91 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 24 May 2023 21:08:25 +0800 Subject: [PATCH 41/59] Add a new line to the reference file --- .../0_stateless/02751_query_log_test_partitions.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.reference b/tests/queries/0_stateless/02751_query_log_test_partitions.reference index 8419e7f9786..5a9f2163c0e 100644 --- a/tests/queries/0_stateless/02751_query_log_test_partitions.reference +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.reference @@ -1,2 +1,2 @@ 3 3 -02751_query_log_test_partitions.3 \ No newline at end of file +02751_query_log_test_partitions.3 From 773a5bbbaae066ac83e35b0215897de949447063 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Fri, 19 May 2023 23:58:32 +0800 Subject: [PATCH 42/59] Optimize predicate with toYear converter The date converters, such as toYear, are widely used in the where clauses of the SQL queries, however, these conversions are often expensive due to the complexity of the calendar system. The function preimage is found an optimization for the predicates with the converters. Given a predicate, toYear(c) = y, we could convert it to its equivalent form: c >= b AND c <= e, where b is "y-01-01" and e is "y-12-31". The similar transformation applies to other comparisons (<>, <, >, <=, <=). This commit implemented the above transformation at the AST level by adding a new pass in the TreeOptimizer and a new AST visitor for in-place replacing the predicates of toYear with the converted ones. --- .../OptimizeDateFilterVisitor.cpp | 122 ++++++++++++++++++ src/Interpreters/OptimizeDateFilterVisitor.h | 20 +++ src/Interpreters/TreeOptimizer.cpp | 19 +++ ..._date_filter_predicate_optimizer.reference | 39 ++++++ .../02764_date_filter_predicate_optimizer.sql | 17 +++ 5 files changed, 217 insertions(+) create mode 100644 src/Interpreters/OptimizeDateFilterVisitor.cpp create mode 100644 src/Interpreters/OptimizeDateFilterVisitor.h create mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference create mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp new file mode 100644 index 00000000000..4c714751f7d --- /dev/null +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -0,0 +1,122 @@ +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +{ + const DateLUTImpl & date_lut = DateLUT::instance(); + + if (converter != "toYear") return {}; + + UInt64 year = compare_to; + String from_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String to_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + + if (comparator == "equals") + { + return makeASTFunction("and", + makeASTFunction("greaterOrEquals", + std::make_shared(column), + std::make_shared(from_date) + ), + makeASTFunction("lessOrEquals", + std::make_shared(column), + std::make_shared(to_date) + ) + ); + } + else if (comparator == "notEquals") + { + return makeASTFunction("or", + makeASTFunction("less", + std::make_shared(column), + std::make_shared(from_date) + ), + makeASTFunction("greater", + std::make_shared(column), + std::make_shared(to_date) + ) + ); + } + else if (comparator == "less" || comparator == "greaterOrEquals") + { + return makeASTFunction(comparator, + std::make_shared(column), + std::make_shared(from_date) + ); + } + else + { + return makeASTFunction(comparator, + std::make_shared(column), + std::make_shared(to_date) + ); + } +} + +bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) +{ + const static std::unordered_map swap_relations = { + {"equals", "equals"}, + {"notEquals", "notEquals"}, + {"less", "greater"}, + {"greater", "less"}, + {"lessOrEquals", "greaterOrEquals"}, + {"greaterOrEquals", "lessOrEquals"}, + }; + + if (!swap_relations.contains(function.name)) return false; + + if (!function.arguments || function.arguments->children.size() != 2) return false; + + size_t func_id = function.arguments->children.size(); + + for (size_t i = 0; i < function.arguments->children.size(); i++) + { + if (const auto * func = function.arguments->children[i]->as(); func) + { + if (func->name == "toYear") + { + func_id = i; + } + } + } + + if (func_id == function.arguments->children.size()) return false; + + size_t literal_id = 1 - func_id; + const auto * literal = function.arguments->children[literal_id]->as(); + + if (!literal || literal->value.getType() != Field::Types::UInt64) return false; + + UInt64 compare_to = literal->value.get(); + String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); + + const auto * func = function.arguments->children[func_id]->as(); + const auto * column_id = func->arguments->children.at(0)->as(); + + if (!column_id) return false; + + String column = column_id->name(); + + const auto new_ast = generateOptimizedDateFilterAST(comparator, func->name, column, compare_to); + + if (!new_ast) return false; + + ast = new_ast; + return true; +} + +void OptimizeDateFilterInPlaceData::visit(ASTFunction & function, ASTPtr & ast) const +{ + rewritePredicateInPlace(function, ast); +} +} diff --git a/src/Interpreters/OptimizeDateFilterVisitor.h b/src/Interpreters/OptimizeDateFilterVisitor.h new file mode 100644 index 00000000000..84394372901 --- /dev/null +++ b/src/Interpreters/OptimizeDateFilterVisitor.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +class ASTFunction; + +/// Rewrite the predicates in place +class OptimizeDateFilterInPlaceData +{ +public: + using TypeToVisit = ASTFunction; + void visit(ASTFunction & function, ASTPtr & ast) const; +}; + +using OptimizeDateFilterInPlaceMatcher = OneTypeMatcher; +using OptimizeDateFilterInPlaceVisitor = InDepthNodeVisitor; +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c38b3c79026..825114b20b7 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -677,6 +678,21 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } +void optimizeDateFilters(ASTSelectQuery * select_query) +{ + /// Predicates in HAVING clause has been moved to WHERE clause. + if (select_query->where()) + { + OptimizeDateFilterInPlaceVisitor::Data data; + OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refWhere()); + } + if (select_query->prewhere()) + { + OptimizeDateFilterInPlaceVisitor::Data data; + OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refPrewhere()); + } +} + void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -780,6 +796,9 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } + /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, toISOWeek, etc. + optimizeDateFilters(select_query); + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference new file mode 100644 index 00000000000..e5c608ddc1a --- /dev/null +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -0,0 +1,39 @@ +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 < \'1993-01-01\') OR (date1 > \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 > \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 <= \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1997-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) OR ((date1 >= \'1994-01-01\') AND (date1 <= \'1994-12-31\'))) AND ((id >= 1) AND (id <= 3)) +SELECT + value1, + toYear(date1) AS year1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +PREWHERE (date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\') +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql new file mode 100644 index 00000000000..563468d4f82 --- /dev/null +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; + +DROP TABLE t; From 802b63f2ab244bcdf6a5c86f541fae9a12599ada Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 11 May 2023 15:44:15 +0800 Subject: [PATCH 43/59] Add 'initial_query_id' field for system.processors_profile_log Facilitate profile data association and aggregation for the same query --- src/Interpreters/ProcessorsProfileLog.cpp | 2 ++ src/Interpreters/ProcessorsProfileLog.h | 1 + src/Interpreters/executeQuery.cpp | 1 + 3 files changed, 4 insertions(+) diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index f6ce801605a..e78a07bb752 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -29,6 +29,7 @@ NamesAndTypesList ProcessorProfileLogElement::getNamesAndTypes() {"plan_step", std::make_shared()}, {"plan_group", std::make_shared()}, + {"initial_query_id", std::make_shared()}, {"query_id", std::make_shared()}, {"name", std::make_shared(std::make_shared())}, {"elapsed_us", std::make_shared()}, @@ -60,6 +61,7 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(plan_step); columns[i++]->insert(plan_group); + columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insertData(processor_name.data(), processor_name.size()); columns[i++]->insert(elapsed_us); diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index 07837bdd10e..81d58edd913 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -19,6 +19,7 @@ struct ProcessorProfileLogElement UInt64 plan_step{}; UInt64 plan_group{}; + String initial_query_id; String query_id; String processor_name; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9a2750f399c..37437410b3c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1003,6 +1003,7 @@ static std::tuple executeQueryImpl( ProcessorProfileLogElement processor_elem; processor_elem.event_time = elem.event_time; processor_elem.event_time_microseconds = elem.event_time_microseconds; + processor_elem.initial_query_id = elem.client_info.initial_query_id; processor_elem.query_id = elem.client_info.current_query_id; auto get_proc_id = [](const IProcessor & proc) -> UInt64 From 515d30caeab92babf612213118cd7f257c5b2bf4 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 17:01:00 +0800 Subject: [PATCH 44/59] Add test case. --- .../en/operations/system-tables/processors_profile_log.md | 8 +++++--- .../0_stateless/02210_processors_profile_log_2.reference | 1 + .../queries/0_stateless/02210_processors_profile_log_2.sh | 2 ++ 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index a6ff15642a1..5eedb5a5dae 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -5,16 +5,18 @@ This table contains profiling on processors level (that you can find in [`EXPLAI Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened. -- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the event happened. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time with microseconds precision when the event happened. - `id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of processor - `parent_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Parent processors IDs +- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step. +- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result. +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query - `name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the processor. - `elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was executed. - `input_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting for data (from other processor). - `output_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting because output port was full. -- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step. -- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result. - `input_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows consumed by processor. - `input_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes consumed by processor. - `output_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows generated by processor. diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.reference b/tests/queries/0_stateless/02210_processors_profile_log_2.reference index 5467c7ef2ba..3d0948105d2 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.reference @@ -9,3 +9,4 @@ NullSource 0 0 0 0 NumbersMt 0 0 1000000 8000000 Resize 1 8 1 8 Resize 1 8 1 8 +1 \ No newline at end of file diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.sh b/tests/queries/0_stateless/02210_processors_profile_log_2.sh index 93eabc2f0fe..044954a4e96 100755 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.sh +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.sh @@ -17,3 +17,5 @@ EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -q "select name, sum(input_rows), sum(input_bytes), sum(output_rows), sum(output_bytes) from system.processors_profile_log where query_id = '${QUERY_ID}' group by name, plan_step, plan_group order by name, sum(input_rows), sum(input_bytes), sum(output_rows), sum(output_bytes)" + +${CLICKHOUSE_CLIENT} -q "select countDistinct(initial_query_id) from system.processors_profile_log where query_id = '${QUERY_ID}'" \ No newline at end of file From cb130a1eb3f3069f1f7153e2ad3488dc06f93d2e Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 24 May 2023 20:41:05 +0800 Subject: [PATCH 45/59] Add a new line to the reference file --- .../0_stateless/02210_processors_profile_log_2.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.reference b/tests/queries/0_stateless/02210_processors_profile_log_2.reference index 3d0948105d2..b9a848131fd 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.reference @@ -9,4 +9,4 @@ NullSource 0 0 0 0 NumbersMt 0 0 1000000 8000000 Resize 1 8 1 8 Resize 1 8 1 8 -1 \ No newline at end of file +1 From 1bc4eb1a6c26db08b6af33745feb83f92f093e9b Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Thu, 25 May 2023 13:47:03 +0800 Subject: [PATCH 46/59] OptimizeDateFilterVisitor: Revise variable names for clarity --- .../OptimizeDateFilterVisitor.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index 4c714751f7d..58e1b3335f9 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,26 +10,25 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) { const DateLUTImpl & date_lut = DateLUT::instance(); if (converter != "toYear") return {}; - UInt64 year = compare_to; - String from_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String to_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); if (comparator == "equals") { return makeASTFunction("and", makeASTFunction("greaterOrEquals", std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ), makeASTFunction("lessOrEquals", std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ) ); } @@ -38,11 +37,11 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & return makeASTFunction("or", makeASTFunction("less", std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ), makeASTFunction("greater", std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ) ); } @@ -50,14 +49,14 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & { return makeASTFunction(comparator, std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ); } else { return makeASTFunction(comparator, std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ); } } From 9582d9e892d5bfd7b0c6b46b11f758c15f5ab0cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 09:57:32 +0200 Subject: [PATCH 47/59] Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) MSAn report: ==38627==WARNING: MemorySanitizer: use-of-uninitialized-value 0 0x555599f5e114 in std::__1::__unique_if::__unique_single std::__1::make_unique[abi:v15000]<> build_docker/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:714:32 1 0x555599f5e114 in DB::ClientBase::initOutputFormat() build_docker/./src/Client/ClientBase.cpp:604:21 2 0x555599f590a8 in DB::ClientBase::onData() build_docker/./src/Client/ClientBase.cpp:446:5 3 0x555599f6f36e in DB::ClientBase::receiveAndProcessPacket() build_docker/./src/Client/ClientBase.cpp:1019:17 4 0x555599f6e863 in DB::ClientBase::receiveResult() build_docker/./src/Client/ClientBase.cpp:987:18 5 0x555599f6c05b in DB::ClientBase::processOrdinaryQuery() build_docker/./src/Client/ClientBase.cpp:905:13 6 0x555599f67e05 in DB::ClientBase::processParsedSingleQuery() build_docker/./src/Client/ClientBase.cpp:1711:13 7 0x555599f86fb6 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1975:21 Uninitialized value was created by a heap allocation 8 0x55559bd3e038 in DB::ParserExplainQuery::parseImpl(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&) build_docker/./src/Parsers/ParserExplainQuery.cpp:53:26 9 0x55559bce31f4 in DB::IParserBase::parse(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&)::$_0::operator()() const build_docker/./src/Parsers/IParserBase.cpp:13:20 .. 21 0x55559be13b5c in DB::parseQueryAndMovePosition(DB::IParser&, char const*&, char const*, std::__1::basic_string, std::__1::allocator > const&, bool, unsigned long, unsigned long) build_docker/./src/Parsers/parseQuery.cpp:357:18 22 0x555599f5673a in DB::ClientBase::parseQuery(char const*&, char const*, bool) const build_docker/./src/Client/ClientBase.cpp:362:15 23 0x555599f84a4f in DB::ClientBase::analyzeMultiQueryText() build_docker/./src/Client/ClientBase.cpp:1821:24 24 0x555599f867b3 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1910:22 25 0x555599f8a2fd in DB::ClientBase::processQueryText(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:2120:12 26 0x555599f94aee in DB::ClientBase::runNonInteractive() build_docker/./src/Client/ClientBase.cpp:2403:9 Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.h | 4 ++-- .../02767_into_outfile_extensions_msan.reference | 2 ++ .../0_stateless/02767_into_outfile_extensions_msan.sh | 11 +++++++++++ 3 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference create mode 100755 tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 09f08772468..7db021405e7 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -15,8 +15,8 @@ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; - bool is_into_outfile_with_stdout; - bool is_outfile_append; + bool is_into_outfile_with_stdout = false; + bool is_outfile_append = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference new file mode 100644 index 00000000000..0c8b489c2fd --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference @@ -0,0 +1,2 @@ +Expression ((Projection + Before ORDER BY)) + ReadFromStorage (SystemNumbers) diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh new file mode 100755 index 00000000000..d1e7312b150 --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out" +# only EXPLAIN triggers the problem under MSan +$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'" +cat "$out" +rm -f "$out" From c053d75741aead3764e3be9d955f496dc47749d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:09:33 +0200 Subject: [PATCH 48/59] Fix formatting of INTO OUTFILE extensions (APPEND / AND STDOUT) Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.cpp | 7 +++++++ ...8_into_outfile_extensions_format.reference | 20 +++++++++++++++++++ .../02768_into_outfile_extensions_format.sh | 12 +++++++++++ 3 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/02768_into_outfile_extensions_format.reference create mode 100755 tests/queries/0_stateless/02768_into_outfile_extensions_format.sh diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 9f771ab92e3..5f717715a69 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -35,6 +35,13 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : ""); out_file->formatImpl(s, state, frame); + + s.ostr << (s.hilite ? hilite_keyword : ""); + if (is_outfile_append) + s.ostr << " APPEND"; + if (is_into_outfile_with_stdout) + s.ostr << " AND STDOUT"; + s.ostr << (s.hilite ? hilite_none : ""); } if (format) diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference new file mode 100644 index 00000000000..4ebc1da8865 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference @@ -0,0 +1,20 @@ +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' AND STDOUT +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND AND STDOUT +; + diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh new file mode 100755 index 00000000000..756488076f9 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo " +select * from numbers(1) into outfile '/dev/null'; +select * from numbers(1) into outfile '/dev/null' and stdout; +select * from numbers(1) into outfile '/dev/null' append; +select * from numbers(1) into outfile '/dev/null' append and stdout; +" | clickhouse-format -n From b30cfe55030fd819d0dfa1da62ed74ab29ea9e63 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:14:47 +0200 Subject: [PATCH 49/59] Fix UB in ASTWatchQuery for is_watch_events Signed-off-by: Azat Khuzhin --- src/Parsers/ASTWatchQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTWatchQuery.h b/src/Parsers/ASTWatchQuery.h index 29dadd71675..156fe6828bc 100644 --- a/src/Parsers/ASTWatchQuery.h +++ b/src/Parsers/ASTWatchQuery.h @@ -23,7 +23,7 @@ class ASTWatchQuery : public ASTQueryWithTableAndOutput public: ASTPtr limit_length; - bool is_watch_events; + bool is_watch_events = false; ASTWatchQuery() = default; String getID(char) const override { return "WatchQuery_" + getDatabase() + "_" + getTable(); } From b680697cced2d9a5dabe87219db8b5e75d4867e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:16:54 +0200 Subject: [PATCH 50/59] Initialize POD members of ASTs to make it less error-prone The cost of initializing members is insignificant in compare to parsing, while the cost of the error is high. Signed-off-by: Azat Khuzhin --- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ASTDictionaryAttributeDeclaration.h | 8 ++++---- src/Parsers/ASTOrderByElement.h | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 2008e4f99d1..45814551db8 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -16,7 +16,7 @@ public: std::optional null_modifier; String default_specifier; ASTPtr default_expression; - bool ephemeral_default; + bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; ASTPtr ttl; diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.h b/src/Parsers/ASTDictionaryAttributeDeclaration.h index b6572e89d16..52103650684 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.h +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.h @@ -19,13 +19,13 @@ public: /// Attribute expression ASTPtr expression; /// Is attribute mirrored to the parent identifier - bool hierarchical; + bool hierarchical = false; /// Is hierarchical attribute bidirectional - bool bidirectional; + bool bidirectional = false; /// Flag that shows whether the id->attribute image is injective - bool injective; + bool injective = false; /// MongoDB object ID - bool is_object_id; + bool is_object_id = false; String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); } diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 4d07405c17a..468d2161dff 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -11,14 +11,14 @@ namespace DB class ASTOrderByElement : public IAST { public: - int direction; /// 1 for ASC, -1 for DESC - int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. - bool nulls_direction_was_explicitly_specified; + int direction = 0; /// 1 for ASC, -1 for DESC + int nulls_direction = 0; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. + bool nulls_direction_was_explicitly_specified = false; /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ ASTPtr collation; - bool with_fill; + bool with_fill = false; ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; From 4f5cf656eb7343cc68e053ee9bcb8a7eaa4e7ae9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 09:36:03 +0000 Subject: [PATCH 51/59] Minor adjustment of clickhouse-client/local parameter docs Related to #50210 --- docs/en/interfaces/cli.md | 8 ++++---- docs/en/operations/utilities/clickhouse-local.md | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 6d5395d46e3..f670d464006 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -177,11 +177,11 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--user, -u` – The username. Default value: default. - `--password` – The password. Default value: empty string. - `--ask-password` - Prompt the user to enter a password. -- `--query, -q` – The query to process when using non-interactive mode. You must specify either `query` or `queries-file` option. -- `--queries-file` – file path with queries to execute. You must specify either `query` or `queries-file` option. -- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default). -- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). +- `--query, -q` – The query to process when using non-interactive mode. Cannot be used simultaneously with `--queries-file`. +- `--queries-file` – file path with queries to execute. Cannot be used simultaneously with `--query`. - `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. +- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). +- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default). - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index d6587602990..0443a80cf17 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -183,12 +183,12 @@ Arguments: - `-S`, `--structure` — table structure for input data. - `--input-format` — input format, `TSV` by default. - `-f`, `--file` — path to data, `stdin` by default. -- `-q`, `--query` — queries to execute with `;` as delimiter. You must specify either `query` or `queries-file` option. -- `--queries-file` - file path with queries to execute. You must specify either `query` or `queries-file` option. +- `-q`, `--query` — queries to execute with `;` as delimiter. Cannot be used simultaneously with `--queries-file`. +- `--queries-file` - file path with queries to execute. Cannot be used simultaneously with `--query`. +- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `-N`, `--table` — table name where to put output data, `table` by default. - `--format`, `--output-format` — output format, `TSV` by default. - `-d`, `--database` — default database, `_local` by default. -- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `--stacktrace` — whether to dump debug output in case of exception. - `--echo` — print query before execution. - `--verbose` — more details on query execution. From e4c8c4cecfb80c4afdae961c4ab8a64d42dc84d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BD=95=E6=9D=8E=E5=A4=AB?= Date: Thu, 25 May 2023 17:50:14 +0800 Subject: [PATCH 52/59] Add zookeeper name in endpoint id (#49780) * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. Here is some infomation: 1.old table: CREATE TABLE a1 (`id` UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 2.new table: CREATE TABLE a2 (`id` UInt64) ENGINE = ReplicatedMergeTree('aux1:/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 3.error info: executeQuery: Code: 220. DB::Exception: Duplicate interserver IO endpoint: DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (DUPLICATE_INTERSERVER_IO_ENDPOINT) InterserverIOHTTPHandler: Code: 221. DB::Exception: No interserver IO endpoint named DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (NO_SUCH_INTERSERVER_IO_ENDPOINT) * Revert "Add zookeeper name in endpoint id" This reverts commit 9deb75b249619b7abdd38e3949ca8b3a76c9df8e. * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. * Fix incompatible with a new setting * add a test, fix other issues * Update 02442_auxiliary_zookeeper_endpoint_id.sql * Update 02735_system_zookeeper_connection.reference * Update 02735_system_zookeeper_connection.sql * Update run.sh * Remove the 'no-fasttest' tag * Update 02442_auxiliary_zookeeper_endpoint_id.sql --------- Co-authored-by: Alexander Tokmakov Co-authored-by: Alexander Tokmakov --- docker/test/stateless/run.sh | 3 + docker/test/upgrade/run.sh | 6 ++ src/Common/ZooKeeper/IKeeper.h | 3 +- src/Common/ZooKeeper/TestKeeper.h | 4 +- src/Common/ZooKeeper/ZooKeeper.cpp | 9 ++- src/Common/ZooKeeper/ZooKeeper.h | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 4 +- src/Interpreters/Context.cpp | 6 +- src/Storages/MergeTree/DataPartsExchange.cpp | 18 ++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 66 +++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 2 + .../StorageSystemZooKeeperConnection.cpp | 5 +- tests/config/config.d/database_replicated.xml | 18 +++++ tests/config/config.d/merge_tree.xml | 1 + tests/config/config.d/zookeeper.xml | 9 +++ ..._auxiliary_zookeeper_endpoint_id.reference | 1 + .../02442_auxiliary_zookeeper_endpoint_id.sql | 21 ++++++ ...2735_system_zookeeper_connection.reference | 3 +- .../02735_system_zookeeper_connection.sql | 6 +- 22 files changed, 147 insertions(+), 48 deletions(-) create mode 100644 tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference create mode 100644 tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index e509809c028..5d0a7b50741 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -132,6 +132,9 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') + clickhouse-test "00001_select_1" > /dev/null ||: + clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" ||: + set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 10ba597a33a..bd0c59a12cd 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -65,6 +65,9 @@ sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +# it contains some new settings, but we can safely remove it +rm /etc/clickhouse-server/config.d/merge_tree.xml + start stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log @@ -94,6 +97,9 @@ sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +# it contains some new settings, but we can safely remove it +rm /etc/clickhouse-server/config.d/merge_tree.xml + start clickhouse-client --query="SELECT 'Server version: ', version()" diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 86f9a388644..3eb5819df90 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -466,7 +467,7 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; - virtual String getConnectedAddress() const = 0; + virtual Poco::Net::SocketAddress getConnectedAddress() const = 0; /// If the method will throw an exception, callbacks won't be called. /// diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 11e56daf6b4..4bffa4e1d4f 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -39,7 +39,7 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } - String getConnectedAddress() const override { return connected_zk_address; } + Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void create( @@ -127,7 +127,7 @@ private: zkutil::ZooKeeperArgs args; - String connected_zk_address; + Poco::Net::SocketAddress connected_zk_address; std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index c423e4fd498..a587ad6caf4 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -112,11 +112,10 @@ void ZooKeeper::init(ZooKeeperArgs args_) else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); - String address = impl->getConnectedAddress(); + Poco::Net::SocketAddress address = impl->getConnectedAddress(); - size_t colon_pos = address.find(':'); - connected_zk_host = address.substr(0, colon_pos); - connected_zk_port = address.substr(colon_pos + 1); + connected_zk_host = address.host().toString(); + connected_zk_port = address.port(); connected_zk_index = 0; @@ -124,7 +123,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) { for (size_t i = 0; i < args.hosts.size(); i++) { - if (args.hosts[i] == address) + if (args.hosts[i] == address.toString()) { connected_zk_index = i; break; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 9b85938c726..96f9914b597 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -524,7 +524,7 @@ public: void setServerCompletelyStarted(); String getConnectedZooKeeperHost() const { return connected_zk_host; } - String getConnectedZooKeeperPort() const { return connected_zk_port; } + UInt16 getConnectedZooKeeperPort() const { return connected_zk_port; } size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } private: @@ -591,7 +591,7 @@ private: ZooKeeperArgs args; String connected_zk_host; - String connected_zk_port; + UInt16 connected_zk_port; size_t connected_zk_index; std::mutex mutex; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 34be8aa1332..7f23ac00efe 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -433,7 +433,7 @@ void ZooKeeper::connect( } connected = true; - connected_zk_address = node.address.toString(); + connected_zk_address = node.address; break; } @@ -450,7 +450,7 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - connected_zk_address = ""; + connected_zk_address = Poco::Net::SocketAddress(); message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 6715607ca88..944c5032fac 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -125,7 +125,7 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } - String getConnectedAddress() const override { return connected_zk_address; } + Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void executeGenericRequest( const ZooKeeperRequestPtr & request, @@ -203,7 +203,7 @@ public: private: ACLs default_acls; - String connected_zk_address; + Poco::Net::SocketAddress connected_zk_address; zkutil::ZooKeeperArgs args; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5ec12d7e0ea..b0895939874 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2796,11 +2796,7 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const std::map Context::getAuxiliaryZooKeepers() const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); - - if (!shared->auxiliary_zookeepers.empty()) - return shared->auxiliary_zookeepers; - else - return std::map(); + return shared->auxiliary_zookeepers; } #if USE_ROCKSDB diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f2e35e2dcd2..21fdda28767 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -369,6 +369,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, + const String & zookeeper_name, const String & replica_path, const String & host, int port, @@ -401,13 +402,18 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( /// Validation of the input that may come from malicious replica. auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); + String endpoint_id = getEndpointId( + data_settings->enable_the_endpoint_id_with_zookeeper_name_prefix ? + zookeeper_name + ":" + replica_path : + replica_path); + Poco::URI uri; uri.setScheme(interserver_scheme); uri.setHost(host); uri.setPort(port); uri.setQueryParameters( { - {"endpoint", getEndpointId(replica_path)}, + {"endpoint", endpoint_id}, {"part", part_name}, {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION)}, {"compress", "false"} @@ -630,7 +636,15 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( temporary_directory_lock = {}; /// Try again but without zero-copy - return fetchSelectedPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, + return fetchSelectedPart( + metadata_snapshot, + context, + part_name, + zookeeper_name, + replica_path, + host, + port, + timeouts, user, password, interserver_scheme, throttler, to_detached, tmp_prefix, nullptr, false, disk); } } diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 57fd0d5cff0..704c1b0a226 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -70,6 +70,7 @@ public: const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, + const String & zookeeper_name, const String & replica_path, const String & host, int port, diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c9e81ce9103..5ea99009756 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -159,6 +159,7 @@ struct Settings; M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ M(Bool, allow_vertical_merges_from_compact_to_wide_parts, false, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ + M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ea9ffee4939..7b9a3093e40 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -240,6 +240,15 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl return res; } +String StorageReplicatedMergeTree::getEndpointName() const +{ + const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings(); + if (settings.enable_the_endpoint_id_with_zookeeper_name_prefix) + return zookeeper_name + ":" + replica_path; + + return replica_path; +} + static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); @@ -1841,6 +1850,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che String source_replica_path = fs::path(zookeeper_path) / "replicas" / replica; if (!fetchPart(part_name, metadata_snapshot, + zookeeper_name, source_replica_path, /* to_detached= */ false, entry.quorum, @@ -2341,7 +2351,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) interserver_scheme, address.scheme, address.host); part_desc->res_part = fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path, + metadata_snapshot, getContext(), part_desc->found_new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); @@ -2458,7 +2468,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr interserver_scheme, address.scheme, address.host); return fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), entry.new_part_name, source_replica_path, + metadata_snapshot, getContext(), entry.new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, true); @@ -4042,6 +4052,7 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & bool StorageReplicatedMergeTree::fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, + const String & source_zookeeper_name, const String & source_replica_path, bool to_detached, size_t quorum, @@ -4077,7 +4088,7 @@ bool StorageReplicatedMergeTree::fetchPart( currently_fetching_parts.erase(part_name); }); - LOG_DEBUG(log, "Fetching part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetching part {} from {}:{}", part_name, source_zookeeper_name, source_replica_path); auto settings_ptr = getSettings(); TableLockHolder table_lock_holder; @@ -4134,7 +4145,8 @@ bool StorageReplicatedMergeTree::fetchPart( } else { - LOG_INFO(log, "Not checking checksums of part {} with replica {} because part was removed from ZooKeeper", part_name, source_replica_path); + LOG_INFO(log, "Not checking checksums of part {} with replica {}:{} because part was removed from ZooKeeper", + part_name, source_zookeeper_name, source_replica_path); } } @@ -4187,6 +4199,7 @@ bool StorageReplicatedMergeTree::fetchPart( metadata_snapshot, getContext(), part_name, + source_zookeeper_name, source_replica_path, address.host, address.replication_port, @@ -4279,7 +4292,7 @@ bool StorageReplicatedMergeTree::fetchPart( if (part_to_clone) LOG_DEBUG(log, "Cloned part {} from {}{}", part_name, part_to_clone->name, to_detached ? " (to 'detached' directory)" : ""); else - LOG_DEBUG(log, "Fetched part {} from {}{}", part_name, source_replica_path, to_detached ? " (to 'detached' directory)" : ""); + LOG_DEBUG(log, "Fetched part {} from {}:{}{}", part_name, source_zookeeper_name, source_replica_path, to_detached ? " (to 'detached' directory)" : ""); return true; } @@ -4318,7 +4331,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( currently_fetching_parts.erase(part_name); }); - LOG_DEBUG(log, "Fetching already known part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetching already known part {} from {}:{}", part_name, zookeeper_name, source_replica_path); TableLockHolder table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -4350,7 +4363,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); return fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), part_name, source_replica_path, + metadata_snapshot, getContext(), part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true, @@ -4387,7 +4400,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches); - LOG_DEBUG(log, "Fetched part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetched part {} from {}:{}", part_name, zookeeper_name, source_replica_path); return part->getDataPartStoragePtr(); } @@ -4430,7 +4443,16 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) InterserverIOEndpointPtr data_parts_exchange_ptr = std::make_shared(*this); [[maybe_unused]] auto prev_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, data_parts_exchange_ptr); assert(prev_ptr == nullptr); - getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); + + /// The endpoint id: + /// old format: DataPartsExchange:/clickhouse/tables/default/t1/{shard}/{replica} + /// new format: DataPartsExchange:{zookeeper_name}:/clickhouse/tables/default/t1/{shard}/{replica} + /// Notice: + /// They are incompatible and the default is the old format. + /// If you want to use the new format, please ensure that 'enable_the_endpoint_id_with_zookeeper_name_prefix' of all nodes is true . + /// + getContext()->getInterserverIOHandler().addEndpoint( + data_parts_exchange_ptr->getId(getEndpointName()), data_parts_exchange_ptr); startBeingLeader(); @@ -4555,7 +4577,7 @@ void StorageReplicatedMergeTree::shutdown() auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) { - getContext()->getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_ptr->getId(replica_path)); + getContext()->getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_ptr->getId(getEndpointName())); /// Ask all parts exchange handlers to finish asap. New ones will fail to start data_parts_exchange_ptr->blocker.cancelForever(); /// Wait for all of them @@ -6237,14 +6259,14 @@ void StorageReplicatedMergeTree::fetchPartition( info.table_id = getStorageID(); info.table_id.uuid = UUIDHelpers::Nil; auto expand_from = query_context->getMacros()->expand(from_, info); - String auxiliary_zookeeper_name = zkutil::extractZooKeeperName(expand_from); + String from_zookeeper_name = zkutil::extractZooKeeperName(expand_from); String from = zkutil::extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true); if (from.empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "ZooKeeper path should not be empty"); zkutil::ZooKeeperPtr zookeeper; - if (auxiliary_zookeeper_name != default_zookeeper_name) - zookeeper = getContext()->getAuxiliaryZooKeeper(auxiliary_zookeeper_name); + if (from_zookeeper_name != default_zookeeper_name) + zookeeper = getContext()->getAuxiliaryZooKeeper(from_zookeeper_name); else zookeeper = getZooKeeper(); @@ -6263,12 +6285,12 @@ void StorageReplicatedMergeTree::fetchPartition( */ if (checkIfDetachedPartExists(part_name)) throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Detached part {} already exists.", part_name); - LOG_INFO(log, "Will fetch part {} from shard {} (zookeeper '{}')", part_name, from_, auxiliary_zookeeper_name); + LOG_INFO(log, "Will fetch part {} from shard {}", part_name, from_); try { /// part name , metadata, part_path , true, 0, zookeeper - if (!fetchPart(part_name, metadata_snapshot, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) + if (!fetchPart(part_name, metadata_snapshot, from_zookeeper_name, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) throw Exception(ErrorCodes::UNFINISHED, "Failed to fetch part {} from {}", part_name, from_); } catch (const DB::Exception & e) @@ -6283,7 +6305,7 @@ void StorageReplicatedMergeTree::fetchPartition( } String partition_id = getPartitionIDFromQuery(partition, query_context); - LOG_INFO(log, "Will fetch partition {} from shard {} (zookeeper '{}')", partition_id, from_, auxiliary_zookeeper_name); + LOG_INFO(log, "Will fetch partition {} from shard {}", partition_id, from_); /** Let's check that there is no such partition in the `detached` directory (where we will write the downloaded parts). * Unreliable (there is a race condition) - such a partition may appear a little later. @@ -6307,7 +6329,7 @@ void StorageReplicatedMergeTree::fetchPartition( active_replicas.push_back(replica); if (active_replicas.empty()) - throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No active replicas for shard {}", from); + throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No active replicas for shard {}", from_); /** You must select the best (most relevant) replica. * This is a replica with the maximum `log_pointer`, then with the minimum `queue` size. @@ -6361,7 +6383,8 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_INFO(log, "Some of parts ({}) are missing. Will try to fetch covering parts.", missing_parts.size()); if (try_no >= query_context->getSettings().max_fetch_partition_retries_count) - throw Exception(ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS, "Too many retries to fetch parts from {}", best_replica_path); + throw Exception(ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS, + "Too many retries to fetch parts from {}:{}", from_zookeeper_name, best_replica_path); Strings parts = zookeeper->getChildren(fs::path(best_replica_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, parts); @@ -6382,7 +6405,8 @@ void StorageReplicatedMergeTree::fetchPartition( parts_to_fetch = std::move(parts_to_fetch_partition); if (parts_to_fetch.empty()) - throw Exception(ErrorCodes::PARTITION_DOESNT_EXIST, "Partition {} on {} doesn't exist", partition_id, best_replica_path); + throw Exception(ErrorCodes::PARTITION_DOESNT_EXIST, + "Partition {} on {}:{} doesn't exist", partition_id, from_zookeeper_name, best_replica_path); } else { @@ -6392,7 +6416,7 @@ void StorageReplicatedMergeTree::fetchPartition( if (!containing_part.empty()) parts_to_fetch.push_back(containing_part); else - LOG_WARNING(log, "Part {} on replica {} has been vanished.", missing_part, best_replica_path); + LOG_WARNING(log, "Part {} on replica {}:{} has been vanished.", missing_part, from_zookeeper_name, best_replica_path); } } @@ -6405,7 +6429,7 @@ void StorageReplicatedMergeTree::fetchPartition( try { - fetched = fetchPart(part, metadata_snapshot, best_replica_path, true, 0, zookeeper, /* try_fetch_shared = */ false); + fetched = fetchPart(part, metadata_snapshot, from_zookeeper_name, best_replica_path, true, 0, zookeeper, /* try_fetch_shared = */ false); } catch (const DB::Exception & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 29b6a4d6817..cb93dd0b5e3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -382,6 +382,7 @@ private: zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const; zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const; void setZooKeeper(); + String getEndpointName() const; /// If true, the table is offline and can not be written to it. /// This flag is managed by RestartingThread. @@ -699,6 +700,7 @@ private: bool fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, + const String & source_zookeeper_name, const String & source_replica_path, bool to_detached, size_t quorum, diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index f249097654e..559e12ad5ee 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -13,7 +13,7 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() return { {"name", std::make_shared()}, {"host", std::make_shared()}, - {"port", std::make_shared()}, + {"port", std::make_shared()}, {"index", std::make_shared()}, {"connected_time", std::make_shared()}, {"is_expired", std::make_shared()}, @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - res_columns[0]->insert("default_zookeeper"); + res_columns[0]->insert("default"); res_columns[1]->insert(context->getZooKeeper()->getConnectedZooKeeperHost()); res_columns[2]->insert(context->getZooKeeper()->getConnectedZooKeeperPort()); res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); @@ -38,7 +38,6 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co { res_columns[0]->insert(elem.first); res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); - res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 3fea87c4fd1..9a405f85908 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -14,6 +14,24 @@ + + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + /test/chroot/auxiliary_zookeeper2 + + + 9181 1 diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml index 43bdb6aa07b..bf2da9b09a2 100644 --- a/tests/config/config.d/merge_tree.xml +++ b/tests/config/config.d/merge_tree.xml @@ -1,5 +1,6 @@ + 1 8 diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 63057224ef9..75b4a00fe67 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -7,4 +7,13 @@ 9181 + + + + localhost + 9181 + + /test/chroot/auxiliary_zookeeper2 + + diff --git a/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference new file mode 100644 index 00000000000..4a9341ba3f6 --- /dev/null +++ b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference @@ -0,0 +1 @@ +10013 diff --git a/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql new file mode 100644 index 00000000000..ff3552b2a42 --- /dev/null +++ b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql @@ -0,0 +1,21 @@ +-- Tags: no-fasttest + +drop table if exists t1_r1 sync; +drop table if exists t1_r2 sync; +drop table if exists t2 sync; + +create table t1_r1 (x Int32) engine=ReplicatedMergeTree('/test/02442/{database}/t', 'r1') order by x; + +create table t1_r2 (x Int32) engine=ReplicatedMergeTree('/test/02442/{database}/t', 'r2') order by x; + +-- create table with same replica_path as t1_r1 +create table t2 (x Int32) engine=ReplicatedMergeTree('zookeeper2:/test/02442/{database}/t', 'r1') order by x; +drop table t2 sync; + +-- insert data into one replica +insert into t1_r1 select * from generateRandom('x Int32') LIMIT 10013; +system sync replica t1_r2; +select count() from t1_r2; + +drop table t1_r1 sync; +drop table t1_r2 sync; diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index c9cc8adede8..1deabd88b88 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1 +1,2 @@ -[ :1]:9181 0 +default ::1 9181 0 0 3 +zookeeper2 ::1 9181 0 0 0 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index b409913d183..8b37c428413 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -5,9 +5,11 @@ DROP TABLE IF EXISTS test_zk_connection_table; CREATE TABLE test_zk_connection_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') +ENGINE ReplicatedMergeTree('zookeeper2:/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') ORDER BY tuple(); -select host, port, is_expired from system.zookeeper_connection where name='default_zookeeper'; +-- keeper_api_version will by 0 for auxiliary_zookeeper2, because we fail to get /api_version due to chroot +-- I'm not sure if it's a bug or a useful trick to fallback to basic api +select name, host, port, index, is_expired, keeper_api_version from system.zookeeper_connection order by name; DROP TABLE IF EXISTS test_zk_connection_table; From 8804dfd4b052bf768d2747fec814c1956f7f40f4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 11:55:23 +0000 Subject: [PATCH 53/59] Fix resizing --- src/Functions/space.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 9d825a8b294..7d55d704038 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -27,9 +27,9 @@ private: static constexpr auto space = ' '; /// Safety threshold against DoS. - static inline void checkRepeatTime(UInt64 repeat_time) + static inline void checkRepeatTime(size_t repeat_time) { - static constexpr UInt64 max_repeat_times = 1'000'000; + static constexpr auto max_repeat_times = 1'000'000uz; if (repeat_time > max_repeat_times) throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } @@ -114,8 +114,8 @@ public: checkRepeatTime(times); - if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.resize(2 * res_chars.capacity()); + if (pos + times + 1 > res_chars.size()) + res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); memset(res_chars.begin() + pos, space, times); pos += times; From 1c3b6738f426779391e5126d34cd8f7d6cdaccfe Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 May 2023 14:41:04 +0200 Subject: [PATCH 54/59] Fixes for parallel replicas (#50195) --- src/Client/HedgedConnections.cpp | 2 +- src/Client/MultiplexedConnections.cpp | 2 +- src/Core/Settings.h | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 52 +++++++++++++------ src/Interpreters/InterpreterSelectQuery.h | 2 + src/Interpreters/interpretSubquery.cpp | 2 +- src/Planner/Planner.cpp | 22 +++++--- .../QueryPlan/ReadFromMergeTree.cpp | 26 +++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 14 +++-- src/Storages/StorageDistributed.cpp | 5 -- src/Storages/StorageMerge.cpp | 15 +----- src/Storages/StorageMerge.h | 1 - tests/broken_tests.txt | 3 ++ .../02535_max_parallel_replicas_custom_key.sh | 2 +- ...lel_replicas_with_final_modifier.reference | 1 - ..._parallel_replicas_with_final_modifier.sql | 5 -- ...rallel_replicas_plain_merge_tree.reference | 0 ...764_parallel_replicas_plain_merge_tree.sql | 16 ++++++ ...parallel_replicas_final_modifier.reference | 0 ...02765_parallel_replicas_final_modifier.sql | 14 +++++ 22 files changed, 113 insertions(+), 76 deletions(-) delete mode 100644 tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference delete mode 100644 tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql create mode 100644 tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference create mode 100644 tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql create mode 100644 tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference create mode 100644 tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fe3acd7cc7b..f8966847e5a 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -174,7 +174,7 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) { diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 7df767a4bcf..71f536b9687 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -142,7 +142,7 @@ void MultiplexedConnections::sendQuery( } } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; size_t num_replicas = replica_states.size(); if (num_replicas > 1) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8967c55b3e2..47d4ba452f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -154,7 +154,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a7a064a66d6..c594feb73c2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4324,7 +4324,7 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const if (!settings_.parallel_replicas_custom_key.value.empty()) return CUSTOM_KEY; - if (settings_.allow_experimental_parallel_reading_from_replicas + if (settings_.allow_experimental_parallel_reading_from_replicas > 0 && !settings_.use_hedged_requests) return READ_TASKS; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e02ce347de..ebaf88ea5d5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -116,6 +116,7 @@ namespace ErrorCodes extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -409,6 +410,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( ApplyWithSubqueryVisitor().visit(query_ptr); } + query_info.query = query_ptr->clone(); query_info.original_query = query_ptr->clone(); if (settings.count_distinct_optimization) @@ -456,25 +458,35 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + /// Check support for JOINs for parallel replicas + if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING(log, "Joins are not supported with parallel replicas. Query will be executed without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - context->setSetting("parallel_replicas_custom_key", String{""}); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "JOINs are not supported with parallel replicas. Query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); + } } - /// Try to execute query without parallel replicas if we find that there is a FINAL modifier there. - bool is_query_with_final = false; - if (query_info.table_expression_modifiers) - is_query_with_final = query_info.table_expression_modifiers->hasFinal(); - else if (query_info.query) - is_query_with_final = query_info.query->as().final(); - - if (is_query_with_final && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + /// Check support for FINAL for parallel replicas + bool is_query_with_final = isQueryWithFinal(query_info); + if (is_query_with_final && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING(log, "FINAL modifier is supported with parallel replicas. Will try to execute the query without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - context->setSetting("parallel_replicas_custom_key", String{""}); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "FINAL modifier is not supported with parallel replicas. Query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "FINAL modifier is not supported with parallel replicas"); + } } /// Rewrite JOINs @@ -3134,4 +3146,14 @@ void InterpreterSelectQuery::initSettings() } } +bool InterpreterSelectQuery::isQueryWithFinal(const SelectQueryInfo & info) +{ + bool result = info.query->as().final(); + if (info.table_expression_modifiers) + result |= info.table_expression_modifiers->hasFinal(); + + return result; +} + + } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 58fddb8ffe9..e39dd675136 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -131,6 +131,8 @@ public: static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context); + static bool isQueryWithFinal(const SelectQueryInfo & info); + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 2358b0ab42a..550fa2912ba 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -113,7 +113,7 @@ std::shared_ptr interpretSubquery( } /// We don't want to execute reading for subqueries in parallel - subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); + subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ba338de4072..3c9df29534d 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -83,6 +83,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } /** ClickHouse query planner. @@ -1192,16 +1193,25 @@ void Planner::buildPlanForQueryNode() const auto & settings = query_context->getSettingsRef(); if (planner_context->getTableExpressionNodeToData().size() > 1 - && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING( - &Poco::Logger::get("Planner"), "Joins are not supported with parallel replicas. Query will be executed without using them."); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING( + &Poco::Logger::get("Planner"), "JOINs are not supported with parallel replicas. Query will be executed without using them."); - auto & mutable_context = planner_context->getMutableQueryContext(); - mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - mutable_context->setSetting("parallel_replicas_custom_key", String{""}); + auto & mutable_context = planner_context->getMutableQueryContext(); + mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + mutable_context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); + } } + /// TODO: Also disable parallel replicas in case of FINAL + auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, select_query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6180a01dec6..b38c3422be1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -99,7 +100,6 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_ROWS; - extern const int SUPPORT_IS_DISABLED; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -1314,7 +1314,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( auto reader_settings = getMergeTreeReaderSettings(context, query_info); bool use_skip_indexes = settings.use_skip_indexes; - bool final = isFinal(query_info); + bool final = InterpreterSelectQuery::isQueryWithFinal(query_info); if (final && !settings.use_skip_indexes_if_final) use_skip_indexes = false; @@ -1377,7 +1377,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (direction != 1 && isFinal(query_info)) + if (direction != 1 && isQueryWithFinal()) return false; auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); @@ -1500,11 +1500,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const bool ReadFromMergeTree::isQueryWithFinal() const { - const auto & select = query_info.query->as(); - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - else - return select.final(); + return InterpreterSelectQuery::isQueryWithFinal(query_info); } bool ReadFromMergeTree::isQueryWithSampling() const @@ -1522,7 +1518,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const Pipe ReadFromMergeTree::spreadMarkRanges( RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) { - bool final = isQueryWithFinal(); + const bool final = isQueryWithFinal(); const auto & input_order_info = query_info.getInputOrderInfo(); Names column_names_to_read = result.column_names_to_read; @@ -1539,8 +1535,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( if (final) { - if (is_parallel_reading_from_replicas) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "FINAL modifier is not supported with parallel replicas"); + chassert(!is_parallel_reading_from_replicas); if (output_each_partition_through_separate_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "Optimisation isn't supposed to be used for queries with final"); @@ -1960,15 +1955,6 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const } } -bool ReadFromMergeTree::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - - const auto & select = query_info.query->as(); - return select.final(); -} - bool MergeTreeDataSelectAnalysisResult::error() const { return std::holds_alternative(result); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5e4ba117967..545ffe84e0c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -159,7 +159,6 @@ public: void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); - static bool isFinal(const SelectQueryInfo & query_info); bool isQueryWithFinal() const; bool isQueryWithSampling() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3d3e7f2c735..1d5ac21f803 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7198,9 +7198,17 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( if (query_context->getClientInfo().collaborate_with_initiator) return QueryProcessingStage::Enum::FetchColumns; - if (query_context->canUseParallelReplicasOnInitiator() - && to_stage >= QueryProcessingStage::WithMergeableState) - return QueryProcessingStage::Enum::WithMergeableState; + /// Parallel replicas + if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) + { + /// ReplicatedMergeTree + if (supportsReplication()) + return QueryProcessingStage::Enum::WithMergeableState; + + /// For non-replicated MergeTree we allow them only if parallel_replicas_for_non_replicated_merge_tree is enabled + if (query_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + return QueryProcessingStage::Enum::WithMergeableState; + } if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86a2599b49c..dd2d3ebfaf0 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -145,7 +145,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INFINITE_LOOP; - extern const int ILLEGAL_FINAL; extern const int TYPE_MISMATCH; extern const int TOO_MANY_ROWS; extern const int UNABLE_TO_SKIP_UNUSED_SHARDS; @@ -1045,10 +1044,6 @@ void StorageDistributed::read( const size_t /*max_block_size*/, const size_t /*num_streams*/) { - const auto * select_query = query_info.query->as(); - if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas) - throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature"); - Block header; ASTPtr query_ast; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7ecb424673d..b0ed242d14d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -633,10 +633,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( auto & modified_select = modified_query_info.query->as(); QueryPipelineBuilderPtr builder; - - bool final = isFinal(modified_query_info); - - if (!final && storage->needRewriteQueryWithFinal(real_column_names)) + if (!InterpreterSelectQuery::isQueryWithFinal(modified_query_info) && storage->needRewriteQueryWithFinal(real_column_names)) { /// NOTE: It may not work correctly in some cases, because query was analyzed without final. /// However, it's needed for MaterializedMySQL and it's unlikely that someone will use it with Merge tables. @@ -1010,21 +1007,13 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) { /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (order_info_->direction != 1 && isFinal(query_info)) + if (order_info_->direction != 1 && InterpreterSelectQuery::isQueryWithFinal(query_info)) return false; order_info = order_info_; return true; } -bool ReadFromMerge::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - const auto & select_query = query_info.query->as(); - return select_query.final(); -} - IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const { ColumnSizeByName column_sizes; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index c4b6d815935..babf0dd92e8 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -145,7 +145,6 @@ public: /// Returns `false` if requested reading cannot be performed. bool requestReadingInOrder(InputOrderInfoPtr order_info_); - static bool isFinal(const SelectQueryInfo & query_info); private: const size_t required_max_block_size; diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index c5a58440a8a..faee1c5b295 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -139,3 +139,6 @@ 02703_row_policy_for_database 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference +02764_parallel_replicas_plain_merge_tree +02765_parallel_replicas_final_modifier + diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 3035a191c8f..50e89cca4c9 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "Joins are not supported with parallel replicas" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with parallel replicas" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference b/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference deleted file mode 100644 index f347e8b5857..00000000000 --- a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference +++ /dev/null @@ -1 +0,0 @@ -1 1 2020-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql b/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql deleted file mode 100644 index 1f5daaae189..00000000000 --- a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS t_02709; -CREATE TABLE t_02709 (key UInt32, sign Int8, date Datetime) ENGINE=CollapsingMergeTree(sign) PARTITION BY date ORDER BY key; -INSERT INTO t_02709 VALUES (1, 1, '2020-01-01'), (2, 1, '2020-01-02'), (1, -1, '2020-01-01'), (2, -1, '2020-01-02'), (1, 1, '2020-01-01'); -SELECT * FROM t_02709 FINAL ORDER BY key SETTINGS max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; -DROP TABLE t_02709; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql new file mode 100644 index 00000000000..ea8eb04bd07 --- /dev/null +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; +INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); + +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; + +SET parallel_replicas_for_non_replicated_merge_tree = 0; + +SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; +SELECT max(length(x)) FROM parallel_replicas_plain FORMAT Null; + +SET parallel_replicas_for_non_replicated_merge_tree = 1; + +SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; +SELECT max(length(x)) FROM parallel_replicas_plain FORMAT Null; + +DROP TABLE IF EXISTS parallel_replicas_plain; diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql new file mode 100644 index 00000000000..f447051e1e5 --- /dev/null +++ b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql @@ -0,0 +1,14 @@ +CREATE TABLE IF NOT EXISTS parallel_replicas_final (x String) ENGINE=ReplacingMergeTree() ORDER BY x; + +INSERT INTO parallel_replicas_final SELECT toString(number) FROM numbers(10); + +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; +SET parallel_replicas_for_non_replicated_merge_tree = 1; + +SELECT * FROM parallel_replicas_final FINAL FORMAT Null; + +SET allow_experimental_parallel_reading_from_replicas=2; + +SELECT * FROM parallel_replicas_final FINAL FORMAT Null; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE IF EXISTS parallel_replicas_final; From 243207aa4e140aba8e708453744e5101b12cf731 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 May 2023 16:00:50 +0200 Subject: [PATCH 55/59] Update S3 sdk to v1.11.61 (#50037) --- contrib/aws | 2 +- contrib/aws-c-auth | 2 +- contrib/aws-c-common | 2 +- contrib/aws-c-event-stream | 2 +- contrib/aws-c-http | 2 +- contrib/aws-c-io | 2 +- contrib/aws-c-mqtt | 2 +- contrib/aws-c-s3 | 2 +- contrib/aws-c-sdkutils | 2 +- contrib/aws-checksums | 2 +- contrib/aws-cmake/CMakeLists.txt | 6 +++--- contrib/aws-crt-cpp | 2 +- contrib/aws-s2n-tls | 2 +- contrib/sparse-checkout/update-aws.sh | 4 ++-- 14 files changed, 17 insertions(+), 17 deletions(-) diff --git a/contrib/aws b/contrib/aws index ecccfc026a4..ca02358dcc7 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit ecccfc026a42b30023289410a67024d561f4bf3e +Subproject commit ca02358dcc7ce3ab733dd4cbcc32734eecfa4ee3 diff --git a/contrib/aws-c-auth b/contrib/aws-c-auth index 30df6c407e2..97133a2b5db 160000 --- a/contrib/aws-c-auth +++ b/contrib/aws-c-auth @@ -1 +1 @@ -Subproject commit 30df6c407e2df43bd244e2c34c9b4a4b87372bfb +Subproject commit 97133a2b5dbca1ccdf88cd6f44f39d0531d27d12 diff --git a/contrib/aws-c-common b/contrib/aws-c-common index 324fd1d973c..45dcb2849c8 160000 --- a/contrib/aws-c-common +++ b/contrib/aws-c-common @@ -1 +1 @@ -Subproject commit 324fd1d973ccb25c813aa747bf1759cfde5121c5 +Subproject commit 45dcb2849c891dba2100b270b4676765c92949ff diff --git a/contrib/aws-c-event-stream b/contrib/aws-c-event-stream index 39bfa94a14b..2f9b60c42f9 160000 --- a/contrib/aws-c-event-stream +++ b/contrib/aws-c-event-stream @@ -1 +1 @@ -Subproject commit 39bfa94a14b7126bf0c1330286ef8db452d87e66 +Subproject commit 2f9b60c42f90840ec11822acda3d8cdfa97a773d diff --git a/contrib/aws-c-http b/contrib/aws-c-http index 2c5a2a7d555..dd344619879 160000 --- a/contrib/aws-c-http +++ b/contrib/aws-c-http @@ -1 +1 @@ -Subproject commit 2c5a2a7d5556600b9782ffa6c9d7e09964df1abc +Subproject commit dd34461987947672444d0bc872c5a733dfdb9711 diff --git a/contrib/aws-c-io b/contrib/aws-c-io index 5d32c453560..d58ed4f272b 160000 --- a/contrib/aws-c-io +++ b/contrib/aws-c-io @@ -1 +1 @@ -Subproject commit 5d32c453560d0823df521a686bf7fbacde7f9be3 +Subproject commit d58ed4f272b1cb4f89ac9196526ceebe5f2b0d89 diff --git a/contrib/aws-c-mqtt b/contrib/aws-c-mqtt index 882c689561a..33c3455cec8 160000 --- a/contrib/aws-c-mqtt +++ b/contrib/aws-c-mqtt @@ -1 +1 @@ -Subproject commit 882c689561a3db1466330ccfe3b63637e0a575d3 +Subproject commit 33c3455cec82b16feb940e12006cefd7b3ef4194 diff --git a/contrib/aws-c-s3 b/contrib/aws-c-s3 index a41255ece72..d7bfe602d69 160000 --- a/contrib/aws-c-s3 +++ b/contrib/aws-c-s3 @@ -1 +1 @@ -Subproject commit a41255ece72a7c887bba7f9d998ca3e14f4c8a1b +Subproject commit d7bfe602d6925948f1fff95784e3613cca6a3900 diff --git a/contrib/aws-c-sdkutils b/contrib/aws-c-sdkutils index 25bf5cf225f..208a701fa01 160000 --- a/contrib/aws-c-sdkutils +++ b/contrib/aws-c-sdkutils @@ -1 +1 @@ -Subproject commit 25bf5cf225f977c3accc6a05a0a7a181ef2a4a30 +Subproject commit 208a701fa01e99c7c8cc3dcebc8317da71362972 diff --git a/contrib/aws-checksums b/contrib/aws-checksums index 48e7c0e0147..ad53be196a2 160000 --- a/contrib/aws-checksums +++ b/contrib/aws-checksums @@ -1 +1 @@ -Subproject commit 48e7c0e01479232f225c8044d76c84e74192889d +Subproject commit ad53be196a25bbefa3700a01187fdce573a7d2d0 diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index 52533cd6483..950a0e06cd0 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,8 +52,8 @@ endif() # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") -SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-core") -SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-s3") +SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/src/aws-cpp-sdk-core") +SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-s3") SET(AWS_AUTH_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-auth") SET(AWS_CAL_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-cal") @@ -118,7 +118,7 @@ configure_file("${AWS_SDK_CORE_DIR}/include/aws/core/SDKConfig.h.in" list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MAJOR=1") list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MINOR=10") list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_PATCH=36") - + list(APPEND AWS_SOURCES ${AWS_SDK_CORE_SRC} ${AWS_SDK_CORE_NET_SRC} ${AWS_SDK_CORE_PLATFORM_SRC}) list(APPEND AWS_PUBLIC_INCLUDES diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index ec0bea288f4..8a301b7e842 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit ec0bea288f451d884c0d80d534bc5c66241c39a4 +Subproject commit 8a301b7e842f1daed478090c869207300972379f diff --git a/contrib/aws-s2n-tls b/contrib/aws-s2n-tls index 0f1ba9e5c4a..71f4794b758 160000 --- a/contrib/aws-s2n-tls +++ b/contrib/aws-s2n-tls @@ -1 +1 @@ -Subproject commit 0f1ba9e5c4a67cb3898de0c0b4f911d4194dc8de +Subproject commit 71f4794b7580cf780eb4aca77d69eded5d3c7bb4 diff --git a/contrib/sparse-checkout/update-aws.sh b/contrib/sparse-checkout/update-aws.sh index c8d4c5a89c2..f86acb54d95 100755 --- a/contrib/sparse-checkout/update-aws.sh +++ b/contrib/sparse-checkout/update-aws.sh @@ -5,8 +5,8 @@ echo "Using sparse checkout for aws" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT echo '!/*/*' >> $FILES_TO_CHECKOUT -echo '/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT -echo '/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT +echo '/src/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT +echo '/generated/src/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 From 4ce722854b4216c1a6e443327bf1a2cd248011b8 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 25 May 2023 11:51:47 -0400 Subject: [PATCH 56/59] Fix URL in backport comment --- tests/ci/cherry_pick.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 211092190fa..2fa562a1386 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -261,7 +261,7 @@ close it. ) self.backport_pr = self.repo.create_pull( title=title, - body=f"Original pull-request {self.pr.url}\n" + body=f"Original pull-request {self.pr.html_url}\n" f"Cherry-pick pull-request #{self.cherrypick_pr.number}\n\n" f"{self.BACKPORT_DESCRIPTION}", base=self.name, From eca08438f42ed43b1e393b2f1dd4b4b1aefefd5e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 17:05:18 +0000 Subject: [PATCH 57/59] Fix macos build --- src/Functions/space.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 7d55d704038..009bc20e065 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -115,7 +115,7 @@ public: checkRepeatTime(times); if (pos + times + 1 > res_chars.size()) - res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); + res_chars.resize(std::max(2 * res_chars.size(), static_cast(pos + times + 1))); memset(res_chars.begin() + pos, space, times); pos += times; From 231d52d0e6d3fc16511522228e4241d95005d15f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 18:27:31 +0000 Subject: [PATCH 58/59] Document system.build_options --- .../operations/system-tables/build_options.md | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 docs/en/operations/system-tables/build_options.md diff --git a/docs/en/operations/system-tables/build_options.md b/docs/en/operations/system-tables/build_options.md new file mode 100644 index 00000000000..5225d0ff99d --- /dev/null +++ b/docs/en/operations/system-tables/build_options.md @@ -0,0 +1,27 @@ +--- +slug: /en/operations/system-tables/build_options +--- +# build_options + +Contains information about the ClickHouse server's build options. + +Columns: + +- `name` (String) — Name of the build option, e.g. `USE_ODBC` +- `value` (String) — Value of the build option, e.g. `1` + +**Example** + +``` sql +SELECT * FROM system.build_options LIMIT 5 +``` + +``` text +┌─name─────────────┬─value─┐ +│ USE_BROTLI │ 1 │ +│ USE_BZIP2 │ 1 │ +│ USE_CAPNP │ 1 │ +│ USE_CASSANDRA │ 1 │ +│ USE_DATASKETCHES │ 1 │ +└──────────────────┴───────┘ +``` From b3a96de533c27283540d0ecad2054a35f86c9357 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 May 2023 00:04:24 +0100 Subject: [PATCH 59/59] Pure parallel replicas: JOIN support (#49544) --- src/Interpreters/ExpressionAnalyzer.cpp | 9 + src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 +- src/Interpreters/InterpreterSelectQuery.cpp | 31 +-- src/Interpreters/interpretSubquery.cpp | 2 - .../02535_max_parallel_replicas_custom_key.sh | 2 +- ...708_parallel_replicas_not_found_column.sql | 1 + ..._parallel_replicas_join_subquery.reference | 44 +++++ .../02731_parallel_replicas_join_subquery.sql | 182 ++++++++++++++++++ ...l_replicas_bug_chunkinfo_not_set.reference | 0 ...arallel_replicas_bug_chunkinfo_not_set.sql | 43 +++++ ...764_parallel_replicas_plain_merge_tree.sql | 2 +- 12 files changed, 312 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 96a86df7ffd..c7c66f6f414 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -969,6 +969,15 @@ const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const return select_query; } +bool ExpressionAnalyzer::isRemoteStorage() const +{ + const Settings & csettings = getContext()->getSettingsRef(); + // Consider any storage used in parallel replicas as remote, so the query is executed in multiple servers + const bool enable_parallel_processing_of_joins + = csettings.max_parallel_replicas > 1 && csettings.allow_experimental_parallel_reading_from_replicas > 0; + return syntax->is_remote_storage || enable_parallel_processing_of_joins; +} + const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const { if (!has_aggregation) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..00cd353aa66 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -201,7 +201,7 @@ protected: const ASTSelectQuery * getSelectQuery() const; - bool isRemoteStorage() const { return syntax->is_remote_storage; } + bool isRemoteStorage() const; NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..08862032007 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -205,10 +205,19 @@ public: } private: + static bool shouldBeExecutedGlobally(const Data & data) + { + const Settings & settings = data.getContext()->getSettingsRef(); + /// For parallel replicas we reinterpret JOIN as GLOBAL JOIN as a way to broadcast data + const bool enable_parallel_processing_of_joins = data.getContext()->canUseParallelReplicasOnInitiator(); + return settings.prefer_global_in_and_join || enable_parallel_processing_of_joins; + } + + /// GLOBAL IN static void visit(ASTFunction & func, ASTPtr &, Data & data) { - if ((data.getContext()->getSettingsRef().prefer_global_in_and_join + if ((shouldBeExecutedGlobally(data) && (func.name == "in" || func.name == "notIn" || func.name == "nullIn" || func.name == "notNullIn")) || func.name == "globalIn" || func.name == "globalNotIn" || func.name == "globalNullIn" || func.name == "globalNotNullIn") { @@ -238,8 +247,7 @@ private: static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data) { if (table_elem.table_join - && (table_elem.table_join->as().locality == JoinLocality::Global - || data.getContext()->getSettingsRef().prefer_global_in_and_join)) + && (table_elem.table_join->as().locality == JoinLocality::Global || shouldBeExecutedGlobally(data))) { data.addExternalStorage(table_elem.table_expression, true); data.has_global_subqueries = true; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ebaf88ea5d5..d4ca2e405e6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -458,19 +458,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - /// Check support for JOINs for parallel replicas - if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) + /// Check support for JOIN for parallel replicas with custom key + if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { - if (settings.allow_experimental_parallel_reading_from_replicas == 1) - { - LOG_WARNING(log, "JOINs are not supported with parallel replicas. Query will be executed without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("parallel_replicas_custom_key", String{""}); - } - else if (settings.allow_experimental_parallel_reading_from_replicas == 2) - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); - } + LOG_WARNING(log, "JOINs are not supported with parallel_replicas_custom_key. Query will be executed without using them."); + context->setSetting("parallel_replicas_custom_key", String{""}); } /// Check support for FINAL for parallel replicas @@ -489,6 +481,21 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// Check support for parallel replicas for non-replicated storage (plain MergeTree) + bool is_plain_merge_tree = storage && storage->isMergeTree() && !storage->supportsReplication(); + if (is_plain_merge_tree && settings.allow_experimental_parallel_reading_from_replicas > 0 && !settings.parallel_replicas_for_non_replicated_merge_tree) + { + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`. For now query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`"); + } + } + /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 550fa2912ba..5f00be07fa5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -112,8 +112,6 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } - /// We don't want to execute reading for subqueries in parallel - subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 50e89cca4c9..9850406eb3a 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with parallel replicas" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql index 8900025502c..ff7b53ce01f 100644 --- a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql +++ b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql @@ -1,3 +1,4 @@ CREATE TABLE IF NOT EXISTS t_02708(x DateTime) ENGINE = MergeTree ORDER BY tuple(); +SET send_logs_level='error'; SELECT count() FROM t_02708 SETTINGS allow_experimental_parallel_reading_from_replicas=1; DROP TABLE t_02708; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference new file mode 100644 index 00000000000..df606679523 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -0,0 +1,44 @@ +=============== INNER QUERY (NO PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== INNER QUERY (PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== +0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +=============== OUTER QUERY (NO PARALLEL) =============== +>T%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +tT%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +t toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` +0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_11888098645495698704_17868075224240210014` USING (`key`) GROUP BY `key`, `value1`, `value2` +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql new file mode 100644 index 00000000000..29c20980c14 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -0,0 +1,182 @@ +-- Tags: zookeeper + +CREATE TABLE join_inner_table +( + id UUID, + key String, + number Int64, + value1 String, + value2 String, + time Int64 +) +ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/join_inner_table', 'r1') +ORDER BY (id, number, key); + +INSERT INTO join_inner_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + rowNumberInAllBlocks()::String as key, + * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) +LIMIT 100; + +SET allow_experimental_analyzer = 0; +SET max_parallel_replicas = 3; +SET prefer_localhost_replica = 1; +SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET use_hedged_requests = 0; +SET joined_subquery_requires_alias = 0; + +SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10; + +SELECT '=============== INNER QUERY (PARALLEL) ==============='; + +-- Parallel inner query alone +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; + +SYSTEM FLUSH LOGS; +-- There should be 4 queries. The main query as received by the initiator and the 3 equal queries sent to each replica +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel inner query alone%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; + +---- Query with JOIN + +CREATE TABLE join_outer_table +( + id UUID, + key String, + otherValue1 String, + otherValue2 String, + time Int64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/join_outer_table', 'r1') +ORDER BY (id, time, key); + +INSERT INTO join_outer_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + (rowNumberInAllBlocks() % 10)::String as key, + * FROM generateRandom('otherValue1 String, otherValue2 String, time Int64', 1, 10, 2) +LIMIT 100; + + +SELECT '=============== OUTER QUERY (NO PARALLEL) ==============='; + +SELECT + value1, + value2, + avg(count) AS avg +FROM +( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 +) +GROUP BY value1, value2 +ORDER BY value1, value2; + +SELECT '=============== OUTER QUERY (PARALLEL) ==============='; + +-- Parallel full query +SELECT + value1, + value2, + avg(count) AS avg +FROM + ( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 + ) +GROUP BY value1, value2 +ORDER BY value1, value2 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SYSTEM FLUSH LOGS; + +-- There should be 7 queries. The main query as received by the initiator, the 3 equal queries to execute the subquery +-- in the inner join and the 3 queries executing the whole query (but replacing the subquery with a temp table) +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel full query%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql new file mode 100644 index 00000000000..2ea2cecc7b5 --- /dev/null +++ b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql @@ -0,0 +1,43 @@ +CREATE TABLE join_inner_table__fuzz_1 +( + `id` UUID, + `key` Nullable(Date), + `number` Int64, + `value1` LowCardinality(String), + `value2` LowCardinality(String), + `time` Int128 +) +ENGINE = MergeTree +ORDER BY (id, number, key) +SETTINGS allow_nullable_key = 1; + +INSERT INTO join_inner_table__fuzz_1 SELECT + CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, + CAST(rowNumberInAllBlocks(), 'String') AS key, + * +FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) +LIMIT 100; + +SET max_parallel_replicas = 3, prefer_localhost_replica = 1, use_hedged_requests = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; + +-- SELECT query will write a Warning to the logs +SET send_logs_level='error'; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table__fuzz_1 +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY + key, + value1, + value2 + WITH ROLLUP +ORDER BY + key ASC, + value1 ASC, + value2 ASC NULLS LAST +LIMIT 10 +FORMAT Null; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index ea8eb04bd07..aaf68dfd300 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; - +SET send_logs_level='error'; SET parallel_replicas_for_non_replicated_merge_tree = 0; SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null;