From 48d47d26a4f18e180fa5602ad0aa89cc7af234a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 14 Sep 2023 14:29:35 +0000 Subject: [PATCH 01/74] Add simple unit test for full sorting join --- .../Transforms/MergeJoinTransform.cpp | 71 +++++++++----- .../Transforms/MergeJoinTransform.h | 30 ++++-- .../tests/gtest_merge_join_algorithm.cpp | 95 +++++++++++++++++++ 3 files changed, 166 insertions(+), 30 deletions(-) create mode 100644 src/Processors/tests/gtest_merge_join_algorithm.cpp diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 159a3244fe9..a9fd7978249 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -270,35 +270,45 @@ bool FullMergeJoinCursor::fullyCompleted() const } MergeJoinAlgorithm::MergeJoinAlgorithm( - JoinPtr table_join_, + JoinKind kind_, + JoinStrictness strictness_, + const TableJoin::JoinOnClause & on_clause_, const Blocks & input_headers, size_t max_block_size_) - : table_join(table_join_) + : kind(kind_) + , strictness(strictness_) , max_block_size(max_block_size_) , log(getLogger("MergeJoinAlgorithm")) { if (input_headers.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinAlgorithm requires exactly two inputs"); - auto strictness = table_join->getTableJoin().strictness(); if (strictness != JoinStrictness::Any && strictness != JoinStrictness::All) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness {}", strictness); - auto kind = table_join->getTableJoin().kind(); if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for kind {}", kind); - const auto & join_on = table_join->getTableJoin().getOnlyClause(); - - if (join_on.on_filter_condition_left || join_on.on_filter_condition_right) + if (on_clause_.on_filter_condition_left || on_clause_.on_filter_condition_right) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); cursors = { - createCursor(input_headers[0], join_on.key_names_left), - createCursor(input_headers[1], join_on.key_names_right) + createCursor(input_headers[0], on_clause_.key_names_left), + createCursor(input_headers[1], on_clause_.key_names_right) }; - for (const auto & [left_key, right_key] : table_join->getTableJoin().leftToRightKeyRemap()) +MergeJoinAlgorithm::MergeJoinAlgorithm( + JoinPtr join_ptr, + const Blocks & input_headers, + size_t max_block_size_) + : MergeJoinAlgorithm( + join_ptr->getTableJoin().kind(), + join_ptr->getTableJoin().strictness(), + join_ptr->getTableJoin().getOnlyClause(), + input_headers, + max_block_size_) +{ + for (const auto & [left_key, right_key] : join_ptr->getTableJoin().leftToRightKeyRemap()) { size_t left_idx = input_headers[0].getPositionByName(left_key); size_t right_idx = input_headers[1].getPositionByName(right_key); @@ -398,7 +408,7 @@ struct AllJoinImpl size_t lnum = nextDistinct(left_cursor.cursor); size_t rnum = nextDistinct(right_cursor.cursor); - bool all_fit_in_block = std::max(left_map.size(), right_map.size()) + lnum * rnum <= max_block_size; + bool all_fit_in_block = !max_block_size || std::max(left_map.size(), right_map.size()) + lnum * rnum <= max_block_size; bool have_all_ranges = left_cursor.cursor.isValid() && right_cursor.cursor.isValid(); if (all_fit_in_block && have_all_ranges) { @@ -498,7 +508,7 @@ std::optional MergeJoinAlgorithm::handleAllJoinState } size_t total_rows = 0; - while (total_rows < max_block_size) + while (!max_block_size || total_rows < max_block_size) { const auto & left_range = all_join_state->getLeft(); const auto & right_range = all_join_state->getRight(); @@ -523,7 +533,7 @@ std::optional MergeJoinAlgorithm::handleAllJoinState return {}; } -MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) +MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin() { PaddedPODArray idx_map[2]; @@ -671,8 +681,6 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState if (any_join_state.empty()) return {}; - auto kind = table_join->getTableJoin().kind(); - Chunk result; for (size_t source_num = 0; source_num < 2; ++source_num) @@ -717,7 +725,7 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState return {}; } -MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) +MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin() { if (auto result = handleAnyJoinState()) return std::move(*result); @@ -804,8 +812,6 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { - auto kind = table_join->getTableJoin().kind(); - if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) return Status(0); @@ -849,13 +855,11 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } } - auto strictness = table_join->getTableJoin().strictness(); - if (strictness == JoinStrictness::Any) - return anyJoin(kind); + return anyJoin(); if (strictness == JoinStrictness::All) - return allJoin(kind); + return allJoin(); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported strictness '{}'", strictness); } @@ -874,9 +878,30 @@ MergeJoinTransform::MergeJoinTransform( /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, table_join, input_headers, max_block_size) +<<<<<<< HEAD , log(getLogger("MergeJoinTransform")) +======= +>>>>>>> b4a16f38320 (Add simple unit test for full sorting join) +{ +} + +MergeJoinTransform::MergeJoinTransform( + JoinKind kind_, + JoinStrictness strictness_, + const TableJoin::JoinOnClause & on_clause_, + const Blocks & input_headers, + const Block & output_header, + size_t max_block_size, + UInt64 limit_hint_) + : IMergingTransform( + input_headers, + output_header, + /* have_all_inputs_= */ true, + limit_hint_, + /* always_read_till_end_= */ false, + /* empty_chunk_on_finish_= */ true, + kind_, strictness_, on_clause_, input_headers, max_block_size) { - LOG_TRACE(log, "Use MergeJoinTransform"); } void MergeJoinTransform::onFinish() diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index cf9331abd59..0b0efa33722 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -19,6 +20,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -227,7 +229,13 @@ private: class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_); + MergeJoinAlgorithm(JoinKind kind_, + JoinStrictness strictness_, + const TableJoin::JoinOnClause & on_clause_, + const Blocks & input_headers, + size_t max_block_size_); + + MergeJoinAlgorithm(JoinPtr join_ptr, const Blocks & input_headers, size_t max_block_size_); const char * getName() const override { return "MergeJoinAlgorithm"; } void initialize(Inputs inputs) override; @@ -238,10 +246,10 @@ public: private: std::optional handleAnyJoinState(); - Status anyJoin(JoinKind kind); + Status anyJoin(); std::optional handleAllJoinState(); - Status allJoin(JoinKind kind); + Status allJoin(); Chunk createBlockWithDefaults(size_t source_num); Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const; @@ -251,11 +259,12 @@ private: std::array cursors; - /// Keep some state to make connection between data in different blocks + /// Keep some state to make handle data from different blocks AnyJoinState any_join_state; std::unique_ptr all_join_state; - JoinPtr table_join; + JoinKind kind; + JoinStrictness strictness; size_t max_block_size; int null_direction_hint = 1; @@ -285,12 +294,19 @@ public: size_t max_block_size, UInt64 limit_hint = 0); + MergeJoinTransform( + JoinKind kind_, + JoinStrictness strictness_, + const TableJoin::JoinOnClause & on_clause_, + const Blocks & input_headers, + const Block & output_header, + size_t max_block_size, + UInt64 limit_hint_ = 0); + String getName() const override { return "MergeJoinTransform"; } protected: void onFinish() override; - - LoggerPtr log; }; } diff --git a/src/Processors/tests/gtest_merge_join_algorithm.cpp b/src/Processors/tests/gtest_merge_join_algorithm.cpp new file mode 100644 index 00000000000..9a8b70efc17 --- /dev/null +++ b/src/Processors/tests/gtest_merge_join_algorithm.cpp @@ -0,0 +1,95 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + + +using namespace DB; + + +QueryPipeline buildJoinPipeline(std::shared_ptr left_source, std::shared_ptr right_source) +{ + Blocks inputs; + inputs.emplace_back(left_source->getPort().getHeader()); + inputs.emplace_back(right_source->getPort().getHeader()); + Block out_header = { + ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "t1.x"), + ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "t2.x"), + }; + + TableJoin::JoinOnClause on_clause; + on_clause.key_names_left = {"x"}; + on_clause.key_names_right = {"x"}; + auto joining = std::make_shared( + JoinKind::Inner, + JoinStrictness::All, + on_clause, + inputs, out_header, /* max_block_size = */ 0); + + chassert(joining->getInputs().size() == 2); + + connect(left_source->getPort(), joining->getInputs().front()); + connect(right_source->getPort(), joining->getInputs().back()); + + auto * output_port = &joining->getOutputPort(); + + auto processors = std::make_shared(); + processors->emplace_back(std::move(left_source)); + processors->emplace_back(std::move(right_source)); + processors->emplace_back(std::move(joining)); + + QueryPipeline pipeline(QueryPlanResourceHolder{}, processors, output_port); + return pipeline; +} + + +std::shared_ptr createSourceWithSingleValue(size_t rows_per_chunk, size_t total_chunks) +{ + Block header = { + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "x") + }; + + Chunks chunks; + + for (size_t i = 0; i < total_chunks; ++i) + { + auto col = ColumnUInt64::create(rows_per_chunk, 1); + chunks.emplace_back(Columns{std::move(col)}, rows_per_chunk); + } + + return std::make_shared(std::move(header), std::move(chunks)); +} + +TEST(FullSortingJoin, Simple) +try +{ + auto left_source = createSourceWithSingleValue(3, 10); + auto right_source = createSourceWithSingleValue(2, 15); + + auto pipeline = buildJoinPipeline(left_source, right_source); + PullingPipelineExecutor executor(pipeline); + + Block block; + + size_t total_result_rows = 0; + while (executor.pull(block)) + { + total_result_rows += block.rows(); + } + ASSERT_EQ(total_result_rows, 3 * 10 * 2 * 15); +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} From 2412f8521985c8d31322ed04baa502c4e7543ef6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 18 Sep 2023 16:14:02 +0000 Subject: [PATCH 02/74] wip full sorting asof join --- .../Transforms/MergeJoinTransform.cpp | 138 ++++++++- .../Transforms/MergeJoinTransform.h | 38 ++- .../tests/gtest_full_sorting_join.cpp | 287 ++++++++++++++++++ .../tests/gtest_merge_join_algorithm.cpp | 95 ------ 4 files changed, 446 insertions(+), 112 deletions(-) create mode 100644 src/Processors/tests/gtest_full_sorting_join.cpp delete mode 100644 src/Processors/tests/gtest_merge_join_algorithm.cpp diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index a9fd7978249..8370e548fcb 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -34,13 +34,15 @@ namespace ErrorCodes namespace { -FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) +constexpr UInt64 DEFAULT_VALUE_INDEX = std::numeric_limits::max(); + +FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, JoinStrictness strictness) { SortDescription desc; desc.reserve(columns.size()); for (const auto & name : columns) desc.emplace_back(name); - return std::make_unique(materializeBlock(block), desc); + return std::make_unique(materializeBlock(block), desc, strictness == JoinStrictness::Asof); } template @@ -90,9 +92,10 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, const SortCursorImpl & rhs, size_t rpos, + size_t key_length, int null_direction_hint) { - for (size_t i = 0; i < lhs.sort_columns_size; ++i) + for (size_t i = 0; i < key_length; ++i) { /// TODO(@vdimir): use nullableCompareAt only if there's nullable columns int cmp = nullableCompareAt(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos, null_direction_hint); @@ -104,13 +107,18 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs, int null_direction_hint) { - return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), null_direction_hint); + return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), lhs.sort_columns_size, null_direction_hint); +} + +int compareAsofCursors(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +{ + return nullableCompareAt(lhs.getAsofColumn(), rhs.getAsofColumn(), lhs->getRow(), rhs->getRow()); } bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) { /// The last row of left cursor is less than the current row of the right cursor. - int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow(), null_direction_hint); + int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow(), lhs.sort_columns_size, null_direction_hint); return cmp < 0; } @@ -222,11 +230,11 @@ Chunk getRowFromChunk(const Chunk & chunk, size_t pos) return result; } -void inline addRange(PaddedPODArray & left_map, size_t start, size_t end) +void inline addRange(PaddedPODArray & values, UInt64 start, UInt64 end) { assert(end > start); - for (size_t i = start; i < end; ++i) - left_map.push_back(i); + for (UInt64 i = start; i < end; ++i) + values.push_back(i); } void inline addMany(PaddedPODArray & left_or_right_map, size_t idx, size_t num) @@ -235,6 +243,11 @@ void inline addMany(PaddedPODArray & left_or_right_map, size_t idx, size left_or_right_map.push_back(idx); } +void inline addMany(PaddedPODArray & values, UInt64 value, size_t num) +{ + values.resize_fill(values.size() + num, value); +} + } const Chunk & FullMergeJoinCursor::getCurrent() const @@ -283,9 +296,15 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( if (input_headers.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinAlgorithm requires exactly two inputs"); - if (strictness != JoinStrictness::Any && strictness != JoinStrictness::All) + if (strictness != JoinStrictness::Any && strictness != JoinStrictness::All && strictness != JoinStrictness::Asof) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness {}", strictness); + if (strictness == JoinStrictness::Asof) + { + if (kind != JoinKind::Left && kind != JoinKind::Inner) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not implement ASOF {} join", kind); + } + if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for kind {}", kind); @@ -293,8 +312,8 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); cursors = { - createCursor(input_headers[0], on_clause_.key_names_left), - createCursor(input_headers[1], on_clause_.key_names_right) + createCursor(input_headers[0], on_clause_.key_names_left, strictness), + createCursor(input_headers[1], on_clause_.key_names_right, strictness), }; MergeJoinAlgorithm::MergeJoinAlgorithm( @@ -313,6 +332,8 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( size_t left_idx = input_headers[0].getPositionByName(left_key); size_t right_idx = input_headers[1].getPositionByName(right_key); left_to_right_key_remap[left_idx] = right_idx; + if (strictness == JoinStrictness::Asof) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ASOF joins USING"); } const auto *smjPtr = typeid_cast(table_join.get()); @@ -321,6 +342,19 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( null_direction_hint = smjPtr->getNullDirection(); } + if (strictness == JoinStrictness::Asof) + setAsofInequality(join_ptr->getTableJoin().getAsofInequality()); +} + +void MergeJoinAlgorithm::setAsofInequality(ASOFJoinInequality asof_inequality_) +{ + if (strictness != JoinStrictness::Asof) + throw Exception(ErrorCodes::LOGICAL_ERROR, "setAsofInequality is only supported for ASOF joins"); + + if (asof_inequality_ == ASOFJoinInequality::None) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ASOF inequality cannot be None"); + + asof_inequality = asof_inequality_; } void MergeJoinAlgorithm::logElapsed(double seconds) @@ -770,6 +804,81 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin() return Status(std::move(result)); } + +MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() +{ + auto & left_cursor = *cursors[0]; + if (!left_cursor->isValid()) + return Status(0); + + auto & right_cursor = *cursors[1]; + if (!right_cursor->isValid()) + return Status(1); + + PaddedPODArray left_map; + PaddedPODArray right_map; + + while (left_cursor->isValid() && right_cursor->isValid()) + { + auto lpos = left_cursor->getRow(); + auto rpos = right_cursor->getRow(); + auto cmp = compareCursors(*left_cursor, *right_cursor); + if (cmp == 0) + { + auto asof_cmp = compareAsofCursors(left_cursor, right_cursor); + if ((asof_inequality == ASOFJoinInequality::Less && asof_cmp <= -1) + || (asof_inequality == ASOFJoinInequality::LessOrEquals && asof_cmp <= 0)) + { + /// First row in right table that is greater (or equal) than current row in left table + /// matches asof join condition the best + left_map.push_back(lpos); + right_map.push_back(rpos); + left_cursor->next(); + continue; + } + + if (asof_inequality == ASOFJoinInequality::Less || asof_inequality == ASOFJoinInequality::LessOrEquals) + { + /// Asof condition is not (yet) satisfied, skip row in right table + right_cursor->next(); + continue; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO: implement ASOF equality join"); + } + else if (cmp < 0) + { + /// no matches for rows in left table, just pass them through + size_t num = nextDistinct(*left_cursor); + if (isLeft(kind)) + { + /// return them with default values at right side + addRange(left_map, lpos, lpos + num); + addMany(right_map, DEFAULT_VALUE_INDEX, num); + } + } + else + { + /// skip rows in right table until we find match for current row in left table + nextDistinct(*right_cursor); + } + } + + chassert(left_map.size() == right_map.size()); + Chunk result; + { + Columns lcols = indexColumns(left_cursor.getCurrent().getColumns(), left_map); + for (auto & col : lcols) + result.addColumn(std::move(col)); + + Columns rcols = indexColumns(right_cursor.getCurrent().getColumns(), right_map); + for (auto & col : rcols) + result.addColumn(std::move(col)); + } + UNUSED(asof_inequality); + return Status(std::move(result)); +} + + /// if `source_num == 0` get data from left cursor and fill defaults at right /// otherwise - vice versa Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const @@ -861,6 +970,9 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (strictness == JoinStrictness::All) return allJoin(); + if (strictness == JoinStrictness::Asof) + return asofJoin(); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported strictness '{}'", strictness); } @@ -878,10 +990,6 @@ MergeJoinTransform::MergeJoinTransform( /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, table_join, input_headers, max_block_size) -<<<<<<< HEAD - , log(getLogger("MergeJoinTransform")) -======= ->>>>>>> b4a16f38320 (Add simple unit test for full sorting join) { } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 0b0efa33722..3ee01e57992 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -195,10 +195,27 @@ private: class FullMergeJoinCursor : boost::noncopyable { public: - explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) + FullMergeJoinCursor( + const Block & sample_block_, + const SortDescription & description_, + bool is_asof = false) : sample_block(sample_block_.cloneEmpty()) , desc(description_) { + if (desc.size() == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty sort description for FullMergeJoinCursor"); + + if (is_asof) + { + /// For ASOF join prefix of sort description is used for equality comparison + /// and the last column is used for inequality comparison and is handled separately + + auto asof_column_description = desc.back(); + desc.pop_back(); + + chassert(asof_column_description.direction == 1 && asof_column_description.nulls_direction == 1); + asof_column_position = sample_block.getPositionByName(asof_column_description.column_name); + } } bool fullyCompleted() const; @@ -209,17 +226,27 @@ public: SortCursorImpl * operator-> () { return &cursor; } const SortCursorImpl * operator-> () const { return &cursor; } + SortCursorImpl & operator* () { return cursor; } + const SortCursorImpl & operator* () const { return cursor; } + SortCursorImpl cursor; const Block & sampleBlock() const { return sample_block; } Columns sampleColumns() const { return sample_block.getColumns(); } + const IColumn & getAsofColumn() const + { + return *cursor.all_columns[asof_column_position]; + } + private: Block sample_block; SortDescription desc; Chunk current_chunk; bool recieved_all_blocks = false; + + size_t asof_column_position; }; /* @@ -242,8 +269,9 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; - void logElapsed(double seconds); + void setAsofInequality(ASOFJoinInequality asof_inequality_); + void logElapsed(double seconds); private: std::optional handleAnyJoinState(); Status anyJoin(); @@ -251,13 +279,17 @@ private: std::optional handleAllJoinState(); Status allJoin(); + Status asofJoin(); + Chunk createBlockWithDefaults(size_t source_num); Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const; + /// For `USING` join key columns should have values from right side instead of defaults std::unordered_map left_to_right_key_remap; std::array cursors; + ASOFJoinInequality asof_inequality = ASOFJoinInequality::None; /// Keep some state to make handle data from different blocks AnyJoinState any_join_state; @@ -305,6 +337,8 @@ public: String getName() const override { return "MergeJoinTransform"; } + void setAsofInequality(ASOFJoinInequality asof_inequality_) { algorithm.setAsofInequality(asof_inequality_); } + protected: void onFinish() override; }; diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp new file mode 100644 index 00000000000..888e280b55f --- /dev/null +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -0,0 +1,287 @@ +#include + + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + + +using namespace DB; + +UInt64 getAndPrintRandomSeed() +{ + UInt64 seed = randomSeed(); + std::cerr << "TEST_RANDOM_SEED: " << seed << std::endl; + return seed; +} + +static UInt64 TEST_RANDOM_SEED = getAndPrintRandomSeed(); + +static pcg64 rng(TEST_RANDOM_SEED); + + +QueryPipeline buildJoinPipeline( + std::shared_ptr left_source, + std::shared_ptr right_source, + size_t key_length = 1, + JoinKind kind = JoinKind::Inner, + JoinStrictness strictness = JoinStrictness::All, + ASOFJoinInequality asof_inequality = ASOFJoinInequality::None) +{ + Blocks inputs; + inputs.emplace_back(left_source->getPort().getHeader()); + inputs.emplace_back(right_source->getPort().getHeader()); + + Block out_header; + for (const auto & input : inputs) + { + for (ColumnWithTypeAndName column : input) + { + if (&input == &inputs.front()) + column.name = "t1." + column.name; + else + column.name = "t2." + column.name; + out_header.insert(column); + } + } + + TableJoin::JoinOnClause on_clause; + for (size_t i = 0; i < key_length; ++i) + { + on_clause.key_names_left.emplace_back(inputs[0].getByPosition(i).name); + on_clause.key_names_right.emplace_back(inputs[1].getByPosition(i).name); + } + + auto joining = std::make_shared( + kind, + strictness, + on_clause, + inputs, out_header, /* max_block_size = */ 0); + + if (asof_inequality != ASOFJoinInequality::None) + joining->setAsofInequality(asof_inequality); + + chassert(joining->getInputs().size() == 2); + + connect(left_source->getPort(), joining->getInputs().front()); + connect(right_source->getPort(), joining->getInputs().back()); + + auto * output_port = &joining->getOutputPort(); + + auto processors = std::make_shared(); + processors->emplace_back(std::move(left_source)); + processors->emplace_back(std::move(right_source)); + processors->emplace_back(std::move(joining)); + + QueryPipeline pipeline(QueryPlanResourceHolder{}, processors, output_port); + return pipeline; +} + + +std::shared_ptr oneColumnSource(const std::vector> & values) +{ + Block header = { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "x") }; + Chunks chunks; + for (const auto & chunk_values : values) + { + auto column = ColumnUInt64::create(); + for (auto n : chunk_values) + column->insertValue(n); + chunks.emplace_back(Chunk(Columns{std::move(column)}, chunk_values.size())); + } + return std::make_shared(header, std::move(chunks)); +} + + +TEST(FullSortingJoin, Simple) +try +{ + auto left_source = oneColumnSource({ {1, 2, 3, 4, 5} }); + auto right_source = oneColumnSource({ {1}, {2}, {3}, {4}, {5} }); + + auto pipeline = buildJoinPipeline(left_source, right_source); + PullingPipelineExecutor executor(pipeline); + + Block block; + + size_t total_result_rows = 0; + while (executor.pull(block)) + total_result_rows += block.rows(); + + ASSERT_EQ(total_result_rows, 5); +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} + +std::shared_ptr sourceFromRows( + const Block & header, const std::vector> & values, double break_prob = 0.0) +{ + Chunks chunks; + auto columns = header.cloneEmptyColumns(); + + std::uniform_real_distribution<> prob_dis(0.0, 1.0); + + + for (auto row : values) + { + if (!columns.empty() && (row.empty() || prob_dis(rng) < break_prob)) + { + size_t rows = columns.front()->size(); + chunks.emplace_back(std::move(columns), rows); + columns = header.cloneEmptyColumns(); + continue; + } + + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->insert(row[i]); + } + + if (!columns.empty()) + chunks.emplace_back(std::move(columns), columns.front()->size()); + + return std::make_shared(header, std::move(chunks)); +} + + +std::vector> getValuesFromBlock(const Block & block, const Names & names) +{ + std::vector> result; + for (size_t i = 0; i < block.rows(); ++i) + { + auto & row = result.emplace_back(); + for (const auto & name : names) + block.getByName(name).column->get(i, row.emplace_back()); + } + return result; +} + + +Block executePipeline(QueryPipeline & pipeline) +{ + PullingPipelineExecutor executor(pipeline); + + Blocks result_blocks; + while (true) + { + Block block; + bool is_ok = executor.pull(block); + if (!is_ok) + break; + result_blocks.emplace_back(std::move(block)); + } + + return concatenateBlocks(result_blocks); +} + +TEST(FullSortingJoin, Asof) +try +{ + const std::vector chunk_break = {}; + + auto left_source = sourceFromRows({ + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "key"), + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), + }, { + {"AMZN", 3}, + {"AMZN", 4}, + {"AMZN", 6}, + }); + + auto right_source = sourceFromRows({ + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "key"), + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "value"), + }, { + {"AAPL", 1, 97}, + chunk_break, + {"AAPL", 2, 98}, + {"AAPL", 3, 99}, + {"AMZN", 1, 100}, + {"AMZN", 2, 110}, + chunk_break, + {"AMZN", 4, 130}, + {"AMZN", 5, 140}, + }); + + auto pipeline = buildJoinPipeline( + left_source, right_source, /* key_length = */ 2, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals); + + Block result_block = executePipeline(pipeline); + auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); + ASSERT_EQ(values.size(), 2); + ASSERT_EQ(values[0], (std::vector{"AMZN", 3u, 4u, 130u})); + ASSERT_EQ(values[1], (std::vector{"AMZN", 4u, 4u, 130u})); +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} + + +TEST(FullSortingJoin, AsofOnlyColumn) +try +{ + const std::vector chunk_break = {}; + + auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} }); + + UInt64 p = std::uniform_int_distribution<>(0, 2)(rng); + double break_prob = p == 0 ? 0.0 : (p == 1 ? 0.5 : 1.0); + + auto right_source = sourceFromRows({ + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), + ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "value"), + }, { + {1, 101}, + {2, 102}, + {4, 104}, + {5, 105}, + {11, 111}, + {15, 115}, + }, + break_prob); + + auto pipeline = buildJoinPipeline( + left_source, right_source, /* key_length = */ 1, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals); + + Block result_block = executePipeline(pipeline); + + ASSERT_EQ( + assert_cast(result_block.getByName("t1.x").column.get())->getData(), + (ColumnUInt64::Container{3, 3, 3, 3, 3, 5, 5, 6, 9, 9, 10}) + ); + + ASSERT_EQ( + assert_cast(result_block.getByName("t2.t").column.get())->getData(), + (ColumnUInt64::Container{4, 4, 4, 4, 4, 5, 5, 11, 11, 11, 15}) + ); + + ASSERT_EQ( + assert_cast(result_block.getByName("t2.value").column.get())->getData(), + (ColumnUInt64::Container{104, 104, 104, 104, 104, 105, 105, 111, 111, 111, 115}) + ); +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} diff --git a/src/Processors/tests/gtest_merge_join_algorithm.cpp b/src/Processors/tests/gtest_merge_join_algorithm.cpp deleted file mode 100644 index 9a8b70efc17..00000000000 --- a/src/Processors/tests/gtest_merge_join_algorithm.cpp +++ /dev/null @@ -1,95 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - - -using namespace DB; - - -QueryPipeline buildJoinPipeline(std::shared_ptr left_source, std::shared_ptr right_source) -{ - Blocks inputs; - inputs.emplace_back(left_source->getPort().getHeader()); - inputs.emplace_back(right_source->getPort().getHeader()); - Block out_header = { - ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "t1.x"), - ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "t2.x"), - }; - - TableJoin::JoinOnClause on_clause; - on_clause.key_names_left = {"x"}; - on_clause.key_names_right = {"x"}; - auto joining = std::make_shared( - JoinKind::Inner, - JoinStrictness::All, - on_clause, - inputs, out_header, /* max_block_size = */ 0); - - chassert(joining->getInputs().size() == 2); - - connect(left_source->getPort(), joining->getInputs().front()); - connect(right_source->getPort(), joining->getInputs().back()); - - auto * output_port = &joining->getOutputPort(); - - auto processors = std::make_shared(); - processors->emplace_back(std::move(left_source)); - processors->emplace_back(std::move(right_source)); - processors->emplace_back(std::move(joining)); - - QueryPipeline pipeline(QueryPlanResourceHolder{}, processors, output_port); - return pipeline; -} - - -std::shared_ptr createSourceWithSingleValue(size_t rows_per_chunk, size_t total_chunks) -{ - Block header = { - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "x") - }; - - Chunks chunks; - - for (size_t i = 0; i < total_chunks; ++i) - { - auto col = ColumnUInt64::create(rows_per_chunk, 1); - chunks.emplace_back(Columns{std::move(col)}, rows_per_chunk); - } - - return std::make_shared(std::move(header), std::move(chunks)); -} - -TEST(FullSortingJoin, Simple) -try -{ - auto left_source = createSourceWithSingleValue(3, 10); - auto right_source = createSourceWithSingleValue(2, 15); - - auto pipeline = buildJoinPipeline(left_source, right_source); - PullingPipelineExecutor executor(pipeline); - - Block block; - - size_t total_result_rows = 0; - while (executor.pull(block)) - { - total_result_rows += block.rows(); - } - ASSERT_EQ(total_result_rows, 3 * 10 * 2 * 15); -} -catch (Exception & e) -{ - std::cout << e.getStackTraceString() << std::endl; - throw; -} From 4079f25a3865f23f23cec9c43aefe241b58e7972 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Sep 2023 12:33:29 +0000 Subject: [PATCH 03/74] fix test --- .../tests/gtest_full_sorting_join.cpp | 35 +++++++++++++------ 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 888e280b55f..b3b462ee1e3 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -134,26 +134,37 @@ std::shared_ptr sourceFromRows( { Chunks chunks; auto columns = header.cloneEmptyColumns(); + chassert(!columns.empty()); std::uniform_real_distribution<> prob_dis(0.0, 1.0); - - for (auto row : values) + for (const auto & row : values) { - if (!columns.empty() && (row.empty() || prob_dis(rng) < break_prob)) + if (!columns.front()->empty() && (row.empty() || prob_dis(rng) < break_prob)) { size_t rows = columns.front()->size(); chunks.emplace_back(std::move(columns), rows); columns = header.cloneEmptyColumns(); - continue; + if (row.empty()) + continue; } + chassert(row.size() == columns.size()); for (size_t i = 0; i < columns.size(); ++i) columns[i]->insert(row[i]); } - if (!columns.empty()) - chunks.emplace_back(std::move(columns), columns.front()->size()); + if (!columns.front()->empty()) + { + size_t rows = columns.front()->size(); + chunks.emplace_back(std::move(columns), rows); + } + + /// Check that code above is correct. + size_t total_result_rows = 0; + for (const auto & chunk : chunks) + total_result_rows += chunk.getNumRows(); + chassert(total_result_rows == values.size()); return std::make_shared(header, std::move(chunks)); } @@ -225,9 +236,11 @@ try Block result_block = executePipeline(pipeline); auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); - ASSERT_EQ(values.size(), 2); - ASSERT_EQ(values[0], (std::vector{"AMZN", 3u, 4u, 130u})); - ASSERT_EQ(values[1], (std::vector{"AMZN", 4u, 4u, 130u})); + + ASSERT_EQ(values, (std::vector>{ + {"AMZN", 3u, 4u, 130u}, + {"AMZN", 4u, 4u, 130u}, + })); } catch (Exception & e) { @@ -272,12 +285,12 @@ try ASSERT_EQ( assert_cast(result_block.getByName("t2.t").column.get())->getData(), - (ColumnUInt64::Container{4, 4, 4, 4, 4, 5, 5, 11, 11, 11, 15}) + (ColumnUInt64::Container{4, 4, 4, 4, 4, 5, 5, 11, 11, 11, 11}) ); ASSERT_EQ( assert_cast(result_block.getByName("t2.value").column.get())->getData(), - (ColumnUInt64::Container{104, 104, 104, 104, 104, 105, 105, 111, 111, 111, 115}) + (ColumnUInt64::Container{104, 104, 104, 104, 104, 105, 105, 111, 111, 111, 111}) ); } catch (Exception & e) From 984d94e5f10e7e1c3cc953e1f5394c153e8fdedc Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Sep 2023 15:46:19 +0000 Subject: [PATCH 04/74] upd gtest_full_sorting_join --- .../tests/gtest_full_sorting_join.cpp | 154 +++++++++--------- 1 file changed, 76 insertions(+), 78 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index b3b462ee1e3..4d7ce25a7e8 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -93,7 +93,7 @@ QueryPipeline buildJoinPipeline( std::shared_ptr oneColumnSource(const std::vector> & values) { - Block header = { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "x") }; + Block header = { ColumnWithTypeAndName(std::make_shared(), "x") }; Chunks chunks; for (const auto & chunk_values : values) { @@ -129,45 +129,46 @@ catch (Exception & e) throw; } -std::shared_ptr sourceFromRows( - const Block & header, const std::vector> & values, double break_prob = 0.0) +class SourceChunksBuilder { +public: + explicit SourceChunksBuilder(const Block & header_) + : header(header_) + { + current_chunk = header.cloneEmptyColumns(); + chassert(!current_chunk.empty()); + } + + SourceChunksBuilder & addRow(const std::vector & row) + { + chassert(row.size() == current_chunk.size()); + for (size_t i = 0; i < current_chunk.size(); ++i) + current_chunk[i]->insert(row[i]); + return *this; + } + + SourceChunksBuilder & addChunk() + { + if (current_chunk.front()->empty()) + return *this; + + size_t rows = current_chunk.front()->size(); + chunks.emplace_back(std::move(current_chunk), rows); + current_chunk = header.cloneEmptyColumns(); + return *this; + } + + std::shared_ptr build() + { + addChunk(); + return std::make_shared(header, std::move(chunks)); + } + +private: + Block header; Chunks chunks; - auto columns = header.cloneEmptyColumns(); - chassert(!columns.empty()); - - std::uniform_real_distribution<> prob_dis(0.0, 1.0); - - for (const auto & row : values) - { - if (!columns.front()->empty() && (row.empty() || prob_dis(rng) < break_prob)) - { - size_t rows = columns.front()->size(); - chunks.emplace_back(std::move(columns), rows); - columns = header.cloneEmptyColumns(); - if (row.empty()) - continue; - } - - chassert(row.size() == columns.size()); - for (size_t i = 0; i < columns.size(); ++i) - columns[i]->insert(row[i]); - } - - if (!columns.front()->empty()) - { - size_t rows = columns.front()->size(); - chunks.emplace_back(std::move(columns), rows); - } - - /// Check that code above is correct. - size_t total_result_rows = 0; - for (const auto & chunk : chunks) - total_result_rows += chunk.getNumRows(); - chassert(total_result_rows == values.size()); - - return std::make_shared(header, std::move(chunks)); -} + MutableColumns current_chunk; +}; std::vector> getValuesFromBlock(const Block & block, const Names & names) @@ -203,32 +204,30 @@ Block executePipeline(QueryPipeline & pipeline) TEST(FullSortingJoin, Asof) try { - const std::vector chunk_break = {}; + auto left_source = SourceChunksBuilder({ + {std::make_shared(), "key"}, + {std::make_shared(), "t"}, + }) + .addRow({"AMZN", 3}) + .addRow({"AMZN", 4}) + .addRow({"AMZN", 6}) + .build(); - auto left_source = sourceFromRows({ - ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "key"), - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), - }, { - {"AMZN", 3}, - {"AMZN", 4}, - {"AMZN", 6}, - }); - - auto right_source = sourceFromRows({ - ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "key"), - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "value"), - }, { - {"AAPL", 1, 97}, - chunk_break, - {"AAPL", 2, 98}, - {"AAPL", 3, 99}, - {"AMZN", 1, 100}, - {"AMZN", 2, 110}, - chunk_break, - {"AMZN", 4, 130}, - {"AMZN", 5, 140}, - }); + auto right_source = SourceChunksBuilder({ + {std::make_shared(), "key"}, + {std::make_shared(), "t"}, + {std::make_shared(), "value"}, + }) + .addRow({"AAPL", 1, 97}) + .addChunk() + .addRow({"AAPL", 2, 98}) + .addRow({"AAPL", 3, 99}) + .addRow({"AMZN", 1, 100}) + .addRow({"AMZN", 2, 110}) + .addChunk() + .addRow({"AMZN", 4, 130}) + .addRow({"AMZN", 5, 140}) + .build(); auto pipeline = buildJoinPipeline( left_source, right_source, /* key_length = */ 2, @@ -252,25 +251,24 @@ catch (Exception & e) TEST(FullSortingJoin, AsofOnlyColumn) try { - const std::vector chunk_break = {}; - auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} }); UInt64 p = std::uniform_int_distribution<>(0, 2)(rng); - double break_prob = p == 0 ? 0.0 : (p == 1 ? 0.5 : 1.0); - auto right_source = sourceFromRows({ - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "t"), - ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "value"), - }, { - {1, 101}, - {2, 102}, - {4, 104}, - {5, 105}, - {11, 111}, - {15, 115}, - }, - break_prob); + SourceChunksBuilder right_source_builder({ + {std::make_shared(), "t"}, + {std::make_shared(), "value"}, + }); + + double break_prob = p == 0 ? 0.0 : (p == 1 ? 0.5 : 1.0); + std::uniform_real_distribution<> prob_dis(0.0, 1.0); + for (const auto & row : std::vector>{ {1, 101}, {2, 102}, {4, 104}, {5, 105}, {11, 111}, {15, 115} }) + { + right_source_builder.addRow(row); + if (prob_dis(rng) < break_prob) + right_source_builder.addChunk(); + } + auto right_source = right_source_builder.build(); auto pipeline = buildJoinPipeline( left_source, right_source, /* key_length = */ 1, From 6330b466aa326e40eb7abca2699554c98241b342 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Sep 2023 11:19:20 +0000 Subject: [PATCH 05/74] Add randomized test FullSortingJoin.AsofGeneratedTestData --- src/Common/getRandomASCIIString.cpp | 7 +- src/Common/getRandomASCIIString.h | 3 + .../Transforms/MergeJoinTransform.cpp | 2 +- .../tests/gtest_full_sorting_join.cpp | 121 +++++++++++++++--- 4 files changed, 116 insertions(+), 17 deletions(-) diff --git a/src/Common/getRandomASCIIString.cpp b/src/Common/getRandomASCIIString.cpp index 594b4cd3228..a295277b453 100644 --- a/src/Common/getRandomASCIIString.cpp +++ b/src/Common/getRandomASCIIString.cpp @@ -6,12 +6,17 @@ namespace DB { String getRandomASCIIString(size_t length) +{ + return getRandomASCIIString(length, thread_local_rng); +} + +String getRandomASCIIString(size_t length, pcg64 & rng) { std::uniform_int_distribution distribution('a', 'z'); String res; res.resize(length); for (auto & c : res) - c = distribution(thread_local_rng); + c = distribution(rng); return res; } diff --git a/src/Common/getRandomASCIIString.h b/src/Common/getRandomASCIIString.h index 627d2700ce3..19e1ff7120e 100644 --- a/src/Common/getRandomASCIIString.h +++ b/src/Common/getRandomASCIIString.h @@ -2,11 +2,14 @@ #include +#include + namespace DB { /// Slow random string. Useful for random names and things like this. Not for generating data. String getRandomASCIIString(size_t length); +String getRandomASCIIString(size_t length, pcg64 & rng); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 8370e548fcb..bfde5892289 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -874,7 +874,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() for (auto & col : rcols) result.addColumn(std::move(col)); } - UNUSED(asof_inequality); + return Status(std::move(result)); } diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 4d7ce25a7e8..741e945bfdb 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -1,30 +1,31 @@ #include - -#include #include #include -#include -#include -#include -#include -#include -#include - #include +#include +#include + #include + #include +#include +#include +#include +#include +#include #include +#include using namespace DB; UInt64 getAndPrintRandomSeed() { UInt64 seed = randomSeed(); - std::cerr << "TEST_RANDOM_SEED: " << seed << std::endl; + std::cerr << __FILE__ << "::" << "TEST_RANDOM_SEED = " << seed << "ull" << std::endl; return seed; } @@ -132,6 +133,8 @@ catch (Exception & e) class SourceChunksBuilder { public: + double break_prob = 0.0; + explicit SourceChunksBuilder(const Block & header_) : header(header_) { @@ -144,6 +147,10 @@ public: chassert(row.size() == current_chunk.size()); for (size_t i = 0; i < current_chunk.size(); ++i) current_chunk[i]->insert(row[i]); + + if (break_prob > 0.0 && std::uniform_real_distribution<>(0.0, 1.0)(rng) < break_prob) + addChunk(); + return *this; } @@ -184,7 +191,7 @@ std::vector> getValuesFromBlock(const Block & block, const Na } -Block executePipeline(QueryPipeline & pipeline) +Block executePipeline(QueryPipeline && pipeline) { PullingPipelineExecutor executor(pipeline); @@ -233,7 +240,7 @@ try left_source, right_source, /* key_length = */ 2, JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals); - Block result_block = executePipeline(pipeline); + Block result_block = executePipeline(std::move(pipeline)); auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); ASSERT_EQ(values, (std::vector>{ @@ -253,13 +260,12 @@ try { auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} }); - UInt64 p = std::uniform_int_distribution<>(0, 2)(rng); - SourceChunksBuilder right_source_builder({ {std::make_shared(), "t"}, {std::make_shared(), "value"}, }); + UInt64 p = std::uniform_int_distribution<>(0, 2)(rng); double break_prob = p == 0 ? 0.0 : (p == 1 ? 0.5 : 1.0); std::uniform_real_distribution<> prob_dis(0.0, 1.0); for (const auto & row : std::vector>{ {1, 101}, {2, 102}, {4, 104}, {5, 105}, {11, 111}, {15, 115} }) @@ -274,7 +280,7 @@ try left_source, right_source, /* key_length = */ 1, JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals); - Block result_block = executePipeline(pipeline); + Block result_block = executePipeline(std::move(pipeline)); ASSERT_EQ( assert_cast(result_block.getByName("t1.x").column.get())->getData(), @@ -296,3 +302,88 @@ catch (Exception & e) std::cout << e.getStackTraceString() << std::endl; throw; } + +TEST(FullSortingJoin, AsofGeneratedTestData) +try +{ + auto left_source_builder = SourceChunksBuilder({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "t"}, + {std::make_shared(), "attr"}, + }); + + auto right_source_builder = SourceChunksBuilder({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "t"}, + {std::make_shared(), "attr"}, + }); + + /// uniform_int_distribution to have 0.0 and 1.0 probabilities + left_source_builder.break_prob = std::uniform_int_distribution<>(0, 5)(rng) / 5.0; + right_source_builder.break_prob = std::uniform_int_distribution<>(0, 5)(rng) / 5.0; + + auto get_next_key = [](UInt64 & k1, String & k2) + { + size_t str_len = std::uniform_int_distribution<>(1, 10)(rng); + String new_k2 = getRandomASCIIString(str_len, rng); + if (new_k2.compare(k2) <= 0) + ++k1; + k2 = new_k2; + }; + + ColumnUInt64::Container expected; + + UInt64 k1 = 0; + String k2 = "asdfg"; + auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); + for (size_t key_num = 0; key_num < key_num_total; ++key_num) + { + UInt64 left_t = 0; + size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); + for (size_t i = 0; i < num_left_rows; ++i) + { + left_t += std::uniform_int_distribution<>(1, 10)(rng); + + left_source_builder.addRow({k1, k2, left_t, 10 * left_t}); + expected.push_back(10 * left_t); + + auto num_matches = 1 + std::poisson_distribution<>(4)(rng); + + size_t right_t = left_t; + for (size_t j = 0; j < num_matches; ++j) + { + right_t += std::uniform_int_distribution<>(0, 3)(rng); + right_source_builder.addRow({k1, k2, right_t, j == 0 ? 100 * left_t : 0}); + } + /// next left_t should be greater than right_t not to match with previous rows + left_t = right_t; + } + + /// generate some rows with greater left_t to check that they are not matched + num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); + for (size_t i = 0; i < num_left_rows; ++i) + { + left_t += std::uniform_int_distribution<>(1, 10)(rng); + left_source_builder.addRow({k1, k2, left_t, 10 * left_t}); + } + + get_next_key(k1, k2); + } + + Block result_block = executePipeline(buildJoinPipeline( + left_source_builder.build(), right_source_builder.build(), + /* key_length = */ 3, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals)); + + ASSERT_EQ(assert_cast(block.getByName("t1.attr").column.get())->getData(), expected); + + for (auto & e : expected) + e = 10 * e; + ASSERT_EQ(assert_cast(block.getByName("t2.attr").column.get())->getData(), expected); +} +catch (Exception & e) { + std::cout << e.getStackTraceString() << std::endl; + throw; +} From da4f35556100847d527f194cee4b2f99b50bfa58 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Sep 2023 11:38:37 +0000 Subject: [PATCH 06/74] upd FullSortingJoin.AsofGeneratedTestData --- .../tests/gtest_full_sorting_join.cpp | 56 +++++++++++++++---- 1 file changed, 44 insertions(+), 12 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 741e945bfdb..49c9f58b1d5 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -208,6 +208,14 @@ Block executePipeline(QueryPipeline && pipeline) return concatenateBlocks(result_blocks); } +template +void checkColumn(const typename ColumnVector::Container & expected, const Block & block, const std::string & name) +{ + const auto & actual = assert_cast *>(block.getByName(name).column.get())->getData(); + EXPECT_EQ(actual.size(), expected.size()); + ASSERT_EQ(actual, expected); +} + TEST(FullSortingJoin, Asof) try { @@ -306,18 +314,27 @@ catch (Exception & e) TEST(FullSortingJoin, AsofGeneratedTestData) try { + std::vector join_kinds = {JoinKind::Inner, JoinKind::Left}; + auto join_kind = join_kinds[std::uniform_int_distribution(0, join_kinds.size() - 1)(rng)]; + + std::vector asof_inequalities = { + ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals, + // ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals, + }; + auto asof_inequality = asof_inequalities[std::uniform_int_distribution(0, asof_inequalities.size() - 1)(rng)]; + auto left_source_builder = SourceChunksBuilder({ {std::make_shared(), "k1"}, {std::make_shared(), "k2"}, {std::make_shared(), "t"}, - {std::make_shared(), "attr"}, + {std::make_shared(), "attr"}, }); auto right_source_builder = SourceChunksBuilder({ {std::make_shared(), "k1"}, {std::make_shared(), "k2"}, {std::make_shared(), "t"}, - {std::make_shared(), "attr"}, + {std::make_shared(), "attr"}, }); /// uniform_int_distribution to have 0.0 and 1.0 probabilities @@ -333,14 +350,14 @@ try k2 = new_k2; }; - ColumnUInt64::Container expected; + ColumnInt64::Container expected; UInt64 k1 = 0; String k2 = "asdfg"; auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { - UInt64 left_t = 0; + Int64 left_t = 0; size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); for (size_t i = 0; i < num_left_rows; ++i) { @@ -351,11 +368,22 @@ try auto num_matches = 1 + std::poisson_distribution<>(4)(rng); - size_t right_t = left_t; + auto right_t = left_t; for (size_t j = 0; j < num_matches; ++j) { - right_t += std::uniform_int_distribution<>(0, 3)(rng); - right_source_builder.addRow({k1, k2, right_t, j == 0 ? 100 * left_t : 0}); + int min_step = 1; + if (asof_inequality == ASOFJoinInequality::LessOrEquals || asof_inequality == ASOFJoinInequality::GreaterOrEquals) + min_step = 0; + right_t += std::uniform_int_distribution<>(min_step, 3)(rng); + + bool is_match = false; + + if (asof_inequality == ASOFJoinInequality::LessOrEquals || asof_inequality == ASOFJoinInequality::Less) + is_match = j == 0; + else if (asof_inequality == ASOFJoinInequality::GreaterOrEquals || asof_inequality == ASOFJoinInequality::Greater) + is_match = j == num_matches - 1; + + right_source_builder.addRow({k1, k2, right_t, is_match ? 100 * left_t : -1}); } /// next left_t should be greater than right_t not to match with previous rows left_t = right_t; @@ -366,7 +394,10 @@ try for (size_t i = 0; i < num_left_rows; ++i) { left_t += std::uniform_int_distribution<>(1, 10)(rng); - left_source_builder.addRow({k1, k2, left_t, 10 * left_t}); + left_source_builder.addRow({k1, k2, left_t, -10 * left_t}); + + if (join_kind == JoinKind::Left) + expected.push_back(-10 * left_t); } get_next_key(k1, k2); @@ -375,13 +406,14 @@ try Block result_block = executePipeline(buildJoinPipeline( left_source_builder.build(), right_source_builder.build(), /* key_length = */ 3, - JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals)); + join_kind, JoinStrictness::Asof, asof_inequality)); - ASSERT_EQ(assert_cast(block.getByName("t1.attr").column.get())->getData(), expected); + checkColumn(expected, result_block, "t1.attr"); for (auto & e : expected) - e = 10 * e; - ASSERT_EQ(assert_cast(block.getByName("t2.attr").column.get())->getData(), expected); + e = e < 0 ? 0 : 10 * e; /// non matched rows from left table have negative attr + + checkColumn(expected, result_block, "t2.attr"); } catch (Exception & e) { std::cout << e.getStackTraceString() << std::endl; From 97e3ee6e661620187bf115d948115936e36cdaf4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Sep 2023 11:40:41 +0000 Subject: [PATCH 07/74] upd gtest_full_sorting_join --- .../tests/gtest_full_sorting_join.cpp | 92 +++++++++++-------- 1 file changed, 54 insertions(+), 38 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 49c9f58b1d5..bbd321a78d7 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -142,7 +142,7 @@ public: chassert(!current_chunk.empty()); } - SourceChunksBuilder & addRow(const std::vector & row) + void addRow(const std::vector & row) { chassert(row.size() == current_chunk.size()); for (size_t i = 0; i < current_chunk.size(); ++i) @@ -150,25 +150,29 @@ public: if (break_prob > 0.0 && std::uniform_real_distribution<>(0.0, 1.0)(rng) < break_prob) addChunk(); - - return *this; } - SourceChunksBuilder & addChunk() + void addChunk() { if (current_chunk.front()->empty()) - return *this; + return; size_t rows = current_chunk.front()->size(); chunks.emplace_back(std::move(current_chunk), rows); current_chunk = header.cloneEmptyColumns(); - return *this; + return; } std::shared_ptr build() { addChunk(); - return std::make_shared(header, std::move(chunks)); + + /// copy chunk to allow reusing same builder + Chunks chunks_copy; + chunks_copy.reserve(chunks.size()); + for (const auto & chunk : chunks) + chunks_copy.emplace_back(chunk.clone()); + return std::make_shared(header, std::move(chunks_copy)); } private: @@ -219,42 +223,54 @@ void checkColumn(const typename ColumnVector::Container & expected, const Blo TEST(FullSortingJoin, Asof) try { - auto left_source = SourceChunksBuilder({ + SourceChunksBuilder left_source({ {std::make_shared(), "key"}, {std::make_shared(), "t"}, - }) - .addRow({"AMZN", 3}) - .addRow({"AMZN", 4}) - .addRow({"AMZN", 6}) - .build(); + }); - auto right_source = SourceChunksBuilder({ + left_source.addRow({"AMZN", 3}); + left_source.addRow({"AMZN", 4}); + left_source.addRow({"AMZN", 6}); + + SourceChunksBuilder right_source({ {std::make_shared(), "key"}, {std::make_shared(), "t"}, {std::make_shared(), "value"}, - }) - .addRow({"AAPL", 1, 97}) - .addChunk() - .addRow({"AAPL", 2, 98}) - .addRow({"AAPL", 3, 99}) - .addRow({"AMZN", 1, 100}) - .addRow({"AMZN", 2, 110}) - .addChunk() - .addRow({"AMZN", 4, 130}) - .addRow({"AMZN", 5, 140}) - .build(); + }); + right_source.addRow({"AAPL", 1, 97}); + right_source.addChunk(); + right_source.addRow({"AAPL", 2, 98}); + right_source.addRow({"AAPL", 3, 99}); + right_source.addRow({"AMZN", 1, 100}); + right_source.addRow({"AMZN", 2, 110}); + right_source.addChunk(); + right_source.addRow({"AMZN", 4, 130}); + right_source.addRow({"AMZN", 5, 140}); - auto pipeline = buildJoinPipeline( - left_source, right_source, /* key_length = */ 2, - JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals); + { + Block result_block = executePipeline(buildJoinPipeline( + left_source.build(), right_source.build(), /* key_length = */ 2, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals)); + auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); - Block result_block = executePipeline(std::move(pipeline)); - auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); + ASSERT_EQ(values, (std::vector>{ + {"AMZN", 3u, 4u, 130u}, + {"AMZN", 4u, 4u, 130u}, + })); + } - ASSERT_EQ(values, (std::vector>{ - {"AMZN", 3u, 4u, 130u}, - {"AMZN", 4u, 4u, 130u}, - })); + { + Block result_block = executePipeline(buildJoinPipeline( + left_source.build(), right_source.build(), /* key_length = */ 2, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals)); + auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); + + ASSERT_EQ(values, (std::vector>{ + {"AMZN", 3u, 2u, 110u}, + {"AMZN", 4u, 4u, 130u}, + {"AMZN", 6u, 5u, 140u}, + })); + } } catch (Exception & e) { @@ -314,23 +330,23 @@ catch (Exception & e) TEST(FullSortingJoin, AsofGeneratedTestData) try { - std::vector join_kinds = {JoinKind::Inner, JoinKind::Left}; + std::array join_kinds{JoinKind::Inner, JoinKind::Left}; auto join_kind = join_kinds[std::uniform_int_distribution(0, join_kinds.size() - 1)(rng)]; - std::vector asof_inequalities = { + std::array asof_inequalities{ ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals, // ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals, }; auto asof_inequality = asof_inequalities[std::uniform_int_distribution(0, asof_inequalities.size() - 1)(rng)]; - auto left_source_builder = SourceChunksBuilder({ + SourceChunksBuilder left_source_builder({ {std::make_shared(), "k1"}, {std::make_shared(), "k2"}, {std::make_shared(), "t"}, {std::make_shared(), "attr"}, }); - auto right_source_builder = SourceChunksBuilder({ + SourceChunksBuilder right_source_builder({ {std::make_shared(), "k1"}, {std::make_shared(), "k2"}, {std::make_shared(), "t"}, From 4a1a7d4c6278246e097cf0583b401aa8ac3775fa Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Sep 2023 10:54:08 +0000 Subject: [PATCH 08/74] Add randomized test FullSortingJoin.Any --- .../Transforms/MergeJoinTransform.cpp | 28 ++- .../Transforms/MergeJoinTransform.h | 5 + .../tests/gtest_full_sorting_join.cpp | 181 ++++++++++++++---- 3 files changed, 166 insertions(+), 48 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index bfde5892289..f5e277ea8c8 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -630,7 +630,7 @@ struct AnyJoinImpl FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map, - AnyJoinState & state, + AnyJoinState & any_join_state, int null_direction_hint) { assert(enabled); @@ -691,21 +691,21 @@ struct AnyJoinImpl } } - /// Remember index of last joined row to propagate it to next block + /// Remember last joined row to propagate it to next block - state.setValue({}); + any_join_state.setValue({}); if (!left_cursor->isValid()) { - state.set(0, left_cursor.cursor); + any_join_state.set(0, left_cursor.cursor); if (cmp == 0 && isLeft(kind)) - state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); + any_join_state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); } if (!right_cursor->isValid()) { - state.set(1, right_cursor.cursor); + any_join_state.set(1, right_cursor.cursor); if (cmp == 0 && isRight(kind)) - state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); + any_join_state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); } } }; @@ -720,7 +720,6 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState for (size_t source_num = 0; source_num < 2; ++source_num) { auto & current = *cursors[source_num]; - auto & state = any_join_state; if (any_join_state.keys[source_num].equals(current.cursor)) { size_t start_pos = current->getRow(); @@ -728,25 +727,22 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState if (length && isLeft(kind) && source_num == 0) { - if (state.value) - result = copyChunkResized(current.getCurrent(), state.value, start_pos, length); + if (any_join_state.value) + result = copyChunkResized(current.getCurrent(), any_join_state.value, start_pos, length); else result = createBlockWithDefaults(source_num, start_pos, length); } if (length && isRight(kind) && source_num == 1) { - if (state.value) - result = copyChunkResized(state.value, current.getCurrent(), start_pos, length); + if (any_join_state.value) + result = copyChunkResized(any_join_state.value, current.getCurrent(), start_pos, length); else result = createBlockWithDefaults(source_num, start_pos, length); } - /// We've found row with other key, no need to skip more rows with current key if (current->isValid()) - { - state.keys[source_num].reset(); - } + any_join_state.keys[source_num].reset(); } else { diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 3ee01e57992..375c9ebd3cc 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -85,6 +85,11 @@ public: keys[source_num] = JoinKeyRow(cursor, cursor.rows - 1); } + void reset(size_t source_num) + { + keys[source_num].reset(); + } + void setValue(Chunk value_) { value = std::move(value_); } bool empty() const { return keys[0].row.empty() && keys[1].row.empty(); } diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index bbd321a78d7..1dc410bd6b8 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -213,13 +213,142 @@ Block executePipeline(QueryPipeline && pipeline) } template -void checkColumn(const typename ColumnVector::Container & expected, const Block & block, const std::string & name) +void assertColumnVectorEq(const typename ColumnVector::Container & expected, const Block & block, const std::string & name) { const auto & actual = assert_cast *>(block.getByName(name).column.get())->getData(); EXPECT_EQ(actual.size(), expected.size()); ASSERT_EQ(actual, expected); } +template +void assertColumnEq(const IColumn & expected, const Block & block, const std::string & name) +{ + const ColumnPtr & actual = block.getByName(name).column; + ASSERT_TRUE(checkColumn(*actual)); + ASSERT_TRUE(checkColumn(expected)); + EXPECT_EQ(actual->size(), expected.size()); + + auto dump_val = [](const IColumn & col, size_t i) -> String + { + Field value; + col.get(i, value); + return value.dump(); + }; + + size_t num_rows = std::min(actual->size(), expected.size()); + for (size_t i = 0; i < num_rows; ++i) + ASSERT_EQ(actual->compareAt(i, i, expected, 1), 0) << dump_val(*actual, i) << " != " << dump_val(expected, i) << " at row " << i; +} + +template +T getRandomFrom(const std::initializer_list & opts) +{ + std::vector options(opts.begin(), opts.end()); + size_t idx = std::uniform_int_distribution(0, options.size() - 1)(rng); + return options[idx]; +} + +/// Used to have accurate 0.0 and 1.0 probabilities +double getRandomDoubleQuantized(size_t quants = 5) +{ + return std::uniform_int_distribution(0, quants)(rng) / static_cast(quants); +} + +void generateNextKey(UInt64 & k1, String & k2) +{ + size_t str_len = std::uniform_int_distribution<>(1, 10)(rng); + String new_k2 = getRandomASCIIString(str_len, rng); + if (new_k2.compare(k2) <= 0) + ++k1; + k2 = new_k2; +} + +TEST(FullSortingJoin, Any) +try +{ + JoinKind kind = getRandomFrom({JoinKind::Inner, JoinKind::Left, JoinKind::Right}); + + SourceChunksBuilder left_source({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "attr"}, + }); + + SourceChunksBuilder right_source({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "attr"}, + }); + + left_source.break_prob = getRandomDoubleQuantized(); + right_source.break_prob = getRandomDoubleQuantized(); + + size_t num_keys = std::uniform_int_distribution<>(100, 1000)(rng); + + auto expected_left = ColumnString::create(); + auto expected_right = ColumnString::create(); + + UInt64 k1 = 0; + String k2 = ""; + + auto get_attr = [&](const String & side, size_t idx) -> String + { + return toString(k1) + "_" + k2 + "_" + side + "_" + toString(idx); + }; + + for (size_t i = 0; i < num_keys; ++i) + { + generateNextKey(k1, k2); + + /// Key is present in left, right or both tables. Both tables is more probable. + size_t key_presence = std::uniform_int_distribution<>(0, 10)(rng); + + size_t num_rows_left = key_presence == 0 ? 0 : std::uniform_int_distribution<>(1, 10)(rng); + for (size_t j = 0; j < num_rows_left; ++j) + left_source.addRow({k1, k2, get_attr("left", j)}); + + size_t num_rows_right = key_presence == 1 ? 0 : std::uniform_int_distribution<>(1, 10)(rng); + for (size_t j = 0; j < num_rows_right; ++j) + right_source.addRow({k1, k2, get_attr("right", j)}); + + String left_attr = num_rows_left ? get_attr("left", 0) : ""; + String right_attr = num_rows_right ? get_attr("right", 0) : ""; + + if (kind == JoinKind::Inner && num_rows_left && num_rows_right) + { + expected_left->insert(left_attr); + expected_right->insert(right_attr); + } + else if (kind == JoinKind::Left) + { + for (size_t j = 0; j < num_rows_left; ++j) + { + expected_left->insert(get_attr("left", j)); + expected_right->insert(right_attr); + } + } + else if (kind == JoinKind::Right) + { + for (size_t j = 0; j < num_rows_right; ++j) + { + expected_left->insert(left_attr); + expected_right->insert(get_attr("right", j)); + } + } + } + + Block result_block = executePipeline(buildJoinPipeline( + left_source.build(), right_source.build(), /* key_length = */ 2, + kind, JoinStrictness::Any)); + assertColumnEq(*expected_left, result_block, "t1.attr"); + assertColumnEq(*expected_right, result_block, "t2.attr"); +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} + TEST(FullSortingJoin, Asof) try { @@ -259,18 +388,18 @@ try })); } - { - Block result_block = executePipeline(buildJoinPipeline( - left_source.build(), right_source.build(), /* key_length = */ 2, - JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals)); - auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); + // { + // Block result_block = executePipeline(buildJoinPipeline( + // left_source.build(), right_source.build(), /* key_length = */ 2, + // JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals)); + // auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); - ASSERT_EQ(values, (std::vector>{ - {"AMZN", 3u, 2u, 110u}, - {"AMZN", 4u, 4u, 130u}, - {"AMZN", 6u, 5u, 140u}, - })); - } + // ASSERT_EQ(values, (std::vector>{ + // {"AMZN", 3u, 2u, 110u}, + // {"AMZN", 4u, 4u, 130u}, + // {"AMZN", 6u, 5u, 140u}, + // })); + // } } catch (Exception & e) { @@ -330,14 +459,12 @@ catch (Exception & e) TEST(FullSortingJoin, AsofGeneratedTestData) try { - std::array join_kinds{JoinKind::Inner, JoinKind::Left}; - auto join_kind = join_kinds[std::uniform_int_distribution(0, join_kinds.size() - 1)(rng)]; + auto join_kind = getRandomFrom({JoinKind::Inner, JoinKind::Left}); - std::array asof_inequalities{ + auto asof_inequality = getRandomFrom({ ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals, // ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals, - }; - auto asof_inequality = asof_inequalities[std::uniform_int_distribution(0, asof_inequalities.size() - 1)(rng)]; + }); SourceChunksBuilder left_source_builder({ {std::make_shared(), "k1"}, @@ -353,18 +480,8 @@ try {std::make_shared(), "attr"}, }); - /// uniform_int_distribution to have 0.0 and 1.0 probabilities - left_source_builder.break_prob = std::uniform_int_distribution<>(0, 5)(rng) / 5.0; - right_source_builder.break_prob = std::uniform_int_distribution<>(0, 5)(rng) / 5.0; - - auto get_next_key = [](UInt64 & k1, String & k2) - { - size_t str_len = std::uniform_int_distribution<>(1, 10)(rng); - String new_k2 = getRandomASCIIString(str_len, rng); - if (new_k2.compare(k2) <= 0) - ++k1; - k2 = new_k2; - }; + left_source_builder.break_prob = getRandomDoubleQuantized(); + right_source_builder.break_prob = getRandomDoubleQuantized(); ColumnInt64::Container expected; @@ -416,7 +533,7 @@ try expected.push_back(-10 * left_t); } - get_next_key(k1, k2); + generateNextKey(k1, k2); } Block result_block = executePipeline(buildJoinPipeline( @@ -424,12 +541,12 @@ try /* key_length = */ 3, join_kind, JoinStrictness::Asof, asof_inequality)); - checkColumn(expected, result_block, "t1.attr"); + assertColumnVectorEq(expected, result_block, "t1.attr"); for (auto & e : expected) e = e < 0 ? 0 : 10 * e; /// non matched rows from left table have negative attr - checkColumn(expected, result_block, "t2.attr"); + assertColumnVectorEq(expected, result_block, "t2.attr"); } catch (Exception & e) { std::cout << e.getStackTraceString() << std::endl; From 7e0c2d7bcb88f6d8575fc1f0e288e94f13456f87 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Sep 2023 12:25:12 +0000 Subject: [PATCH 09/74] wip full sorting asof join --- .../Transforms/MergeJoinTransform.cpp | 300 +++++++++++++-- .../Transforms/MergeJoinTransform.h | 89 ++--- .../tests/gtest_full_sorting_join.cpp | 346 ++++++++++++++---- .../02276_full_sort_join_unsupported.sql | 2 +- 4 files changed, 575 insertions(+), 162 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index f5e277ea8c8..638f2f1cb10 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -34,8 +34,6 @@ namespace ErrorCodes namespace { -constexpr UInt64 DEFAULT_VALUE_INDEX = std::numeric_limits::max(); - FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, JoinStrictness strictness) { SortDescription desc; @@ -112,7 +110,7 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImp int compareAsofCursors(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) { - return nullableCompareAt(lhs.getAsofColumn(), rhs.getAsofColumn(), lhs->getRow(), rhs->getRow()); + return nullableCompareAt(*lhs.getAsofColumn(), *rhs.getAsofColumn(), lhs->getRow(), rhs->getRow()); } bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) @@ -250,6 +248,87 @@ void inline addMany(PaddedPODArray & values, UInt64 value, size_t num) } +JoinKeyRow::JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos) +{ + row.reserve(cursor->sort_columns.size()); + for (const auto & col : cursor->sort_columns) + { + auto new_col = col->cloneEmpty(); + new_col->insertFrom(*col, pos); + row.push_back(std::move(new_col)); + } + if (auto asof_column = cursor.getAsofColumn()) + { + auto new_col = asof_column->cloneEmpty(); + new_col->insertFrom(*asof_column, pos); + row.push_back(std::move(new_col)); + } +} + +void JoinKeyRow::reset() +{ + row.clear(); +} + +bool JoinKeyRow::equals(const FullMergeJoinCursor & cursor) const +{ + if (row.empty()) + return false; + + assert(this->row.size() == cursor->sort_columns_size); + for (size_t i = 0; i < cursor->sort_columns_size; ++i) + { + int cmp = this->row[i]->compareAt(0, cursor->getRow(), *(cursor->sort_columns[i]), cursor->desc[i].nulls_direction); + if (cmp != 0) + return false; + } + return true; +} + +bool JoinKeyRow::asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const +{ + if (!equals(cursor)) + return false; + int cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *row.back(), 1); + return (asof_inequality == ASOFJoinInequality::Less && cmp < 0) + || (asof_inequality == ASOFJoinInequality::LessOrEquals && cmp <= 0) + || (asof_inequality == ASOFJoinInequality::Greater && cmp > 0) + || (asof_inequality == ASOFJoinInequality::GreaterOrEquals && cmp >= 0); +} + +void AnyJoinState::set(size_t source_num, const FullMergeJoinCursor & cursor) +{ + assert(cursor->rows); + keys[source_num] = JoinKeyRow(cursor, cursor->rows - 1); +} + +void AnyJoinState::reset(size_t source_num) +{ + keys[source_num].reset(); + value.clear(); +} + +void AnyJoinState::setValue(Chunk value_) +{ + value = std::move(value_); +} + +bool AnyJoinState::empty() const { return keys[0].row.empty() && keys[1].row.empty(); } + + +void AsofJoinState::set(const FullMergeJoinCursor & rcursor, size_t rpos) +{ + key = JoinKeyRow(rcursor, rpos); + value = rcursor.getCurrent().clone(); + value_row = rpos; +} + +void AsofJoinState::reset() +{ + key.reset(); + value.clear(); +} + const Chunk & FullMergeJoinCursor::getCurrent() const { return current_chunk; @@ -282,6 +361,31 @@ bool FullMergeJoinCursor::fullyCompleted() const return !cursor.isValid() && recieved_all_blocks; } +String FullMergeJoinCursor::dump() const +{ + Strings row_dump; + if (cursor.isValid()) + { + Field val; + for (size_t i = 0; i < cursor.sort_columns_size; ++i) + { + cursor.sort_columns[i]->get(cursor.getRow(), val); + row_dump.push_back(val.dump()); + } + + if (auto * asof_column = getAsofColumn()) + { + asof_column->get(cursor.getRow(), val); + row_dump.push_back(val.dump()); + } + } + + return fmt::format("<{}/{}{}>[{}]", + cursor.getRow(), cursor.rows, + recieved_all_blocks ? "(finished)" : "", + fmt::join(row_dump, ", ")); +} + MergeJoinAlgorithm::MergeJoinAlgorithm( JoinKind kind_, JoinStrictness strictness_, @@ -456,7 +560,7 @@ struct AllJoinImpl else { assert(state == nullptr); - state = std::make_unique(left_cursor.cursor, lpos, right_cursor.cursor, rpos); + state = std::make_unique(left_cursor, lpos, right_cursor, rpos); state->addRange(0, left_cursor.getCurrent().clone(), lpos, lnum); state->addRange(1, right_cursor.getCurrent().clone(), rpos, rnum); return; @@ -501,6 +605,17 @@ void dispatchKind(JoinKind kind, Args && ... args) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); } +MutableColumns MergeJoinAlgorithm::getEmptyResultColumns() const +{ + MutableColumns result_cols; + for (size_t i = 0; i < 2; ++i) + { + for (const auto & col : cursors[i]->sampleColumns()) + result_cols.push_back(col->cloneEmpty()); + } + return result_cols; +} + std::optional MergeJoinAlgorithm::handleAllJoinState() { if (all_join_state && all_join_state->finished()) @@ -514,7 +629,7 @@ std::optional MergeJoinAlgorithm::handleAllJoinState /// Accumulate blocks with same key in all_join_state for (size_t i = 0; i < 2; ++i) { - if (cursors[i]->cursor.isValid() && all_join_state->keys[i].equals(cursors[i]->cursor)) + if (cursors[i]->cursor.isValid() && all_join_state->keys[i].equals(*cursors[i])) { size_t pos = cursors[i]->cursor.getRow(); size_t num = nextDistinct(cursors[i]->cursor); @@ -534,12 +649,7 @@ std::optional MergeJoinAlgorithm::handleAllJoinState stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, all_join_state->blocksStored()); /// join all rows with current key - MutableColumns result_cols; - for (size_t i = 0; i < 2; ++i) - { - for (const auto & col : cursors[i]->sampleColumns()) - result_cols.push_back(col->cloneEmpty()); - } + MutableColumns result_cols = getEmptyResultColumns(); size_t total_rows = 0; while (!max_block_size || total_rows < max_block_size) @@ -567,6 +677,52 @@ std::optional MergeJoinAlgorithm::handleAllJoinState return {}; } +std::optional MergeJoinAlgorithm::handleAsofJoinState() +{ + if (strictness != JoinStrictness::Asof) + return {}; + + if (!cursors[1]->fullyCompleted()) + return {}; + + auto & left_cursor = *cursors[0]; + size_t lpos = left_cursor->getRow(); + const auto & left_columns = left_cursor.getCurrent().getColumns(); + + MutableColumns result_cols = getEmptyResultColumns(); + + while (left_cursor->isValid() && asof_join_state.hasMatch(left_cursor, asof_inequality)) + { + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (const auto & col : asof_join_state.value.getColumns()) + result_cols[i++]->insertFrom(*col, asof_join_state.value_row); + chassert(i == result_cols.size()); + left_cursor->next(); + } + + while (isLeft(kind) && left_cursor->isValid()) + { + /// return row with default values at right side + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (; i < result_cols.size(); ++i) + result_cols[i]->insertDefault(); + chassert(i == result_cols.size()); + + left_cursor->next(); + } + + size_t result_rows = result_cols.empty() ? 0 : result_cols.front()->size(); + if (result_rows) + return Status(Chunk(std::move(result_cols), result_rows)); + + return {}; +} + + MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin() { PaddedPODArray idx_map[2]; @@ -696,14 +852,14 @@ struct AnyJoinImpl any_join_state.setValue({}); if (!left_cursor->isValid()) { - any_join_state.set(0, left_cursor.cursor); + any_join_state.set(0, left_cursor); if (cmp == 0 && isLeft(kind)) any_join_state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); } if (!right_cursor->isValid()) { - any_join_state.set(1, right_cursor.cursor); + any_join_state.set(1, right_cursor); if (cmp == 0 && isRight(kind)) any_join_state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); } @@ -720,7 +876,7 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState for (size_t source_num = 0; source_num < 2; ++source_num) { auto & current = *cursors[source_num]; - if (any_join_state.keys[source_num].equals(current.cursor)) + if (any_join_state.keys[source_num].equals(current)) { size_t start_pos = current->getRow(); size_t length = nextDistinct(current.cursor); @@ -811,24 +967,35 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() if (!right_cursor->isValid()) return Status(1); - PaddedPODArray left_map; - PaddedPODArray right_map; + const auto & left_columns = left_cursor.getCurrent().getColumns(); + const auto & right_columns = right_cursor.getCurrent().getColumns(); + + MutableColumns result_cols = getEmptyResultColumns(); while (left_cursor->isValid() && right_cursor->isValid()) { auto lpos = left_cursor->getRow(); auto rpos = right_cursor->getRow(); auto cmp = compareCursors(*left_cursor, *right_cursor); + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ({}) <=> ({}) -> {}", __FILE__, __LINE__, left_cursor.dump(), right_cursor.dump(), cmp); + if (cmp == 0) { auto asof_cmp = compareAsofCursors(left_cursor, right_cursor); + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ({}) <=> ({}) -> asof {}", __FILE__, __LINE__, left_cursor.dump(), right_cursor.dump(), asof_cmp); + if ((asof_inequality == ASOFJoinInequality::Less && asof_cmp <= -1) || (asof_inequality == ASOFJoinInequality::LessOrEquals && asof_cmp <= 0)) { /// First row in right table that is greater (or equal) than current row in left table /// matches asof join condition the best - left_map.push_back(lpos); - right_map.push_back(rpos); + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (const auto & col : right_columns) + result_cols[i++]->insertFrom(*col, rpos); + chassert(i == result_cols.size()); + left_cursor->next(); continue; } @@ -839,39 +1006,99 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() right_cursor->next(); continue; } + + if ((asof_inequality == ASOFJoinInequality::Greater && asof_cmp >= 1) + || (asof_inequality == ASOFJoinInequality::GreaterOrEquals && asof_cmp >= 0)) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + /// condition is satisfied, remember this row and move next to try to find better match + asof_join_state.set(right_cursor, rpos); + right_cursor->next(); + continue; + } + + if (asof_inequality == ASOFJoinInequality::Greater || asof_inequality == ASOFJoinInequality::GreaterOrEquals) + { + /// Asof condition is not satisfied anymore, use last matched row from right table + if (asof_join_state.hasMatch(left_cursor, asof_inequality)) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (const auto & col : asof_join_state.value.getColumns()) + result_cols[i++]->insertFrom(*col, asof_join_state.value_row); + chassert(i == result_cols.size()); + } + else + { + asof_join_state.reset(); + if (isLeft(kind)) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + + /// return row with default values at right side + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (; i < result_cols.size(); ++i) + result_cols[i]->insertDefault(); + chassert(i == result_cols.size()); + } + } + left_cursor->next(); + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + continue; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO: implement ASOF equality join"); } else if (cmp < 0) { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + if (asof_join_state.hasMatch(left_cursor, asof_inequality)) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertFrom(*col, lpos); + for (const auto & col : asof_join_state.value.getColumns()) + result_cols[i++]->insertFrom(*col, asof_join_state.value_row); + chassert(i == result_cols.size()); + left_cursor->next(); + continue; + } + else + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + asof_join_state.reset(); + } + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + /// no matches for rows in left table, just pass them through size_t num = nextDistinct(*left_cursor); - if (isLeft(kind)) + if (isLeft(kind) && num) { /// return them with default values at right side - addRange(left_map, lpos, lpos + num); - addMany(right_map, DEFAULT_VALUE_INDEX, num); + size_t i = 0; + for (const auto & col : left_columns) + result_cols[i++]->insertRangeFrom(*col, lpos, num); + for (; i < result_cols.size(); ++i) + result_cols[i]->insertManyDefaults(num); + chassert(i == result_cols.size()); } } else { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + /// skip rows in right table until we find match for current row in left table nextDistinct(*right_cursor); } } - - chassert(left_map.size() == right_map.size()); - Chunk result; - { - Columns lcols = indexColumns(left_cursor.getCurrent().getColumns(), left_map); - for (auto & col : lcols) - result.addColumn(std::move(col)); - - Columns rcols = indexColumns(right_cursor.getCurrent().getColumns(), right_map); - for (auto & col : rcols) - result.addColumn(std::move(col)); - } - - return Status(std::move(result)); + size_t num_rows = result_cols.empty() ? 0 : result_cols.front()->size(); + return Status(Chunk(std::move(result_cols), num_rows)); } @@ -929,6 +1156,9 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() return std::move(*result); } + if (auto result = handleAsofJoinState()) + return std::move(*result); + if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 375c9ebd3cc..dbdda0b166b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -37,62 +37,28 @@ using FullMergeJoinCursorPtr = std::unique_ptr; /// Used instead of storing previous block struct JoinKeyRow { - std::vector row; - JoinKeyRow() = default; - explicit JoinKeyRow(const SortCursorImpl & impl_, size_t pos) - { - row.reserve(impl_.sort_columns.size()); - for (const auto & col : impl_.sort_columns) - { - auto new_col = col->cloneEmpty(); - new_col->insertFrom(*col, pos); - row.push_back(std::move(new_col)); - } - } + JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos); - void reset() - { - row.clear(); - } + bool equals(const FullMergeJoinCursor & cursor) const; + bool asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const; - bool equals(const SortCursorImpl & impl) const - { - if (row.empty()) - return false; + void reset(); - assert(this->row.size() == impl.sort_columns_size); - for (size_t i = 0; i < impl.sort_columns_size; ++i) - { - int cmp = this->row[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], impl.desc[i].nulls_direction); - if (cmp != 0) - return false; - } - return true; - } + std::vector row; }; /// Remembers previous key if it was joined in previous block class AnyJoinState : boost::noncopyable { public: - AnyJoinState() = default; + void set(size_t source_num, const FullMergeJoinCursor & cursor); + void setValue(Chunk value_); - void set(size_t source_num, const SortCursorImpl & cursor) - { - assert(cursor.rows); - keys[source_num] = JoinKeyRow(cursor, cursor.rows - 1); - } + void reset(size_t source_num); - void reset(size_t source_num) - { - keys[source_num].reset(); - } - - void setValue(Chunk value_) { value = std::move(value_); } - - bool empty() const { return keys[0].row.empty() && keys[1].row.empty(); } + bool empty() const; /// current keys JoinKeyRow keys[2]; @@ -125,8 +91,8 @@ public: Chunk chunk; }; - AllJoinState(const SortCursorImpl & lcursor, size_t lpos, - const SortCursorImpl & rcursor, size_t rpos) + AllJoinState(const FullMergeJoinCursor & lcursor, size_t lpos, + const FullMergeJoinCursor & rcursor, size_t rpos) : keys{JoinKeyRow(lcursor, lpos), JoinKeyRow(rcursor, rpos)} { } @@ -194,6 +160,25 @@ private: size_t ridx = 0; }; + +class AsofJoinState : boost::noncopyable +{ +public: + void set(const FullMergeJoinCursor & rcursor, size_t rpos); + void reset(); + + bool hasMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) + { + if (value.empty()) + return false; + return key.asofMatch(cursor, asof_inequality); + } + + JoinKeyRow key; + Chunk value; + size_t value_row = 0; +}; + /* * Wrapper for SortCursorImpl */ @@ -239,11 +224,15 @@ public: const Block & sampleBlock() const { return sample_block; } Columns sampleColumns() const { return sample_block.getColumns(); } - const IColumn & getAsofColumn() const + const IColumn * getAsofColumn() const { - return *cursor.all_columns[asof_column_position]; + if (!asof_column_position) + return nullptr; + return cursor.all_columns[*asof_column_position]; } + String dump() const; + private: Block sample_block; SortDescription desc; @@ -251,7 +240,7 @@ private: Chunk current_chunk; bool recieved_all_blocks = false; - size_t asof_column_position; + std::optional asof_column_position; }; /* @@ -284,12 +273,13 @@ private: std::optional handleAllJoinState(); Status allJoin(); + std::optional handleAsofJoinState(); Status asofJoin(); + MutableColumns getEmptyResultColumns() const; Chunk createBlockWithDefaults(size_t source_num); Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const; - /// For `USING` join key columns should have values from right side instead of defaults std::unordered_map left_to_right_key_remap; @@ -299,6 +289,7 @@ private: /// Keep some state to make handle data from different blocks AnyJoinState any_join_state; std::unique_ptr all_join_state; + AsofJoinState asof_join_state; JoinKind kind; JoinStrictness strictness; diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 1dc410bd6b8..4e0727779b7 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -18,19 +18,53 @@ #include #include +#include +#include + + #include using namespace DB; +namespace +{ + +[[ maybe_unused ]] +String dumpBlock(std::shared_ptr source) +{ + WriteBufferFromOwnString buf; + { + Block header = source->getPort().getHeader(); + QueryPipeline pipeline(source); + auto format = std::make_shared(buf, header, FormatSettings{}, false); + pipeline.complete(std::move(format)); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } + return buf.str(); +} + +[[ maybe_unused ]] +String dumpBlock(const Block & block) +{ + Block header = block.cloneEmpty(); + Chunk data(block.getColumns(), block.rows()); + auto source = std::make_shared(header, std::move(data)); + return dumpBlock(std::move(source)); +} + UInt64 getAndPrintRandomSeed() { UInt64 seed = randomSeed(); - std::cerr << __FILE__ << "::" << "TEST_RANDOM_SEED = " << seed << "ull" << std::endl; + if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe) + seed = std::stoull(random_seed); + + std::cerr << __FILE__ << " :: " << "TEST_RANDOM_SEED=" << seed << std::endl; return seed; } static UInt64 TEST_RANDOM_SEED = getAndPrintRandomSeed(); - static pcg64 rng(TEST_RANDOM_SEED); @@ -94,42 +128,29 @@ QueryPipeline buildJoinPipeline( std::shared_ptr oneColumnSource(const std::vector> & values) { - Block header = { ColumnWithTypeAndName(std::make_shared(), "x") }; + Block header = { + ColumnWithTypeAndName(std::make_shared(), "key"), + ColumnWithTypeAndName(std::make_shared(), "idx"), + }; + + UInt64 idx = 0; Chunks chunks; for (const auto & chunk_values : values) { - auto column = ColumnUInt64::create(); + auto key_column = ColumnUInt64::create(); + auto idx_column = ColumnUInt64::create(); + for (auto n : chunk_values) - column->insertValue(n); - chunks.emplace_back(Chunk(Columns{std::move(column)}, chunk_values.size())); + { + key_column->insertValue(n); + idx_column->insertValue(idx); + ++idx; + } + chunks.emplace_back(Chunk(Columns{std::move(key_column), std::move(idx_column)}, chunk_values.size())); } return std::make_shared(header, std::move(chunks)); } - -TEST(FullSortingJoin, Simple) -try -{ - auto left_source = oneColumnSource({ {1, 2, 3, 4, 5} }); - auto right_source = oneColumnSource({ {1}, {2}, {3}, {4}, {5} }); - - auto pipeline = buildJoinPipeline(left_source, right_source); - PullingPipelineExecutor executor(pipeline); - - Block block; - - size_t total_result_rows = 0; - while (executor.pull(block)) - total_result_rows += block.rows(); - - ASSERT_EQ(total_result_rows, 5); -} -catch (Exception & e) -{ - std::cout << e.getStackTraceString() << std::endl; - throw; -} - class SourceChunksBuilder { public: @@ -163,7 +184,7 @@ public: return; } - std::shared_ptr build() + std::shared_ptr getSource() { addChunk(); @@ -215,9 +236,11 @@ Block executePipeline(QueryPipeline && pipeline) template void assertColumnVectorEq(const typename ColumnVector::Container & expected, const Block & block, const std::string & name) { - const auto & actual = assert_cast *>(block.getByName(name).column.get())->getData(); - EXPECT_EQ(actual.size(), expected.size()); - ASSERT_EQ(actual, expected); + const auto * actual = typeid_cast *>(block.getByName(name).column.get()); + ASSERT_TRUE(actual) << "unexpected column type: " << block.getByName(name).column->dumpStructure() << "expected: " << typeid(ColumnVector).name(); + + EXPECT_EQ(actual->getData().size(), expected.size()); + ASSERT_EQ(actual->getData(), expected) << "column name: " << name; } template @@ -263,7 +286,90 @@ void generateNextKey(UInt64 & k1, String & k2) k2 = new_k2; } -TEST(FullSortingJoin, Any) +bool isStrict(ASOFJoinInequality inequality) +{ + return inequality == ASOFJoinInequality::Less || inequality == ASOFJoinInequality::Greater; +} + +} + +TEST(FullSortingJoin, AllAnyOneKey) +try +{ + { + SCOPED_TRACE("Inner All"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {1, 2, 3, 4, 5} }), + oneColumnSource({ {1}, {2}, {3}, {4}, {5} }), + 1, JoinKind::Inner, JoinStrictness::All)); + + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t2.idx"); + } + { + SCOPED_TRACE("Inner Any"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {1, 2, 3, 4, 5} }), + oneColumnSource({ {1}, {2}, {3}, {4}, {5} }), + 1, JoinKind::Inner, JoinStrictness::Any)); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t2.idx"); + } + { + SCOPED_TRACE("Inner All"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }), + oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }), + 1, JoinKind::Inner, JoinStrictness::All)); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 0, 1, 2, 3, 3, 4, 5}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({3, 3, 3, 4, 4, 4, 3, 4, 5, 5}), result, "t2.idx"); + } + { + SCOPED_TRACE("Inner Any"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }), + oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }), + 1, JoinKind::Inner, JoinStrictness::Any)); + assertColumnVectorEq(ColumnUInt64::Container({0, 4}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({3, 5}), result, "t2.idx"); + } + { + SCOPED_TRACE("Inner Any"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {2, 2, 2, 2}, {3}, {3, 5} }), + oneColumnSource({ {1, 1, 1, 2}, {2}, {3, 4} }), + 1, JoinKind::Inner, JoinStrictness::Any)); + assertColumnVectorEq(ColumnUInt64::Container({0, 4}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({3, 5}), result, "t2.idx"); + } + { + + SCOPED_TRACE("Left Any"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }), + oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }), + 1, JoinKind::Left, JoinStrictness::Any)); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4, 5, 6}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({3, 3, 3, 3, 5, 5, 0}), result, "t2.idx"); + } + { + SCOPED_TRACE("Left Any"); + Block result = executePipeline(buildJoinPipeline( + oneColumnSource({ {2, 2, 2, 2}, {3}, {3, 5} }), + oneColumnSource({ {1, 1, 1, 2}, {2}, {3, 4} }), + 1, JoinKind::Left, JoinStrictness::Any)); + assertColumnVectorEq(ColumnUInt64::Container({0, 1, 2, 3, 4, 5, 6}), result, "t1.idx"); + assertColumnVectorEq(ColumnUInt64::Container({3, 3, 3, 3, 5, 5, 0}), result, "t2.idx"); + } +} +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} + + +TEST(FullSortingJoin, AnyRandomized) try { JoinKind kind = getRandomFrom({JoinKind::Inner, JoinKind::Left, JoinKind::Right}); @@ -288,7 +394,7 @@ try auto expected_left = ColumnString::create(); auto expected_right = ColumnString::create(); - UInt64 k1 = 0; + UInt64 k1 = 1; String k2 = ""; auto get_attr = [&](const String & side, size_t idx) -> String @@ -338,7 +444,7 @@ try } Block result_block = executePipeline(buildJoinPipeline( - left_source.build(), right_source.build(), /* key_length = */ 2, + left_source.getSource(), right_source.getSource(), /* key_length = */ 2, kind, JoinStrictness::Any)); assertColumnEq(*expected_left, result_block, "t1.attr"); assertColumnEq(*expected_right, result_block, "t2.attr"); @@ -356,10 +462,10 @@ try {std::make_shared(), "key"}, {std::make_shared(), "t"}, }); - left_source.addRow({"AMZN", 3}); left_source.addRow({"AMZN", 4}); left_source.addRow({"AMZN", 6}); + left_source.addRow({"SBUX", 10}); SourceChunksBuilder right_source({ {std::make_shared(), "key"}, @@ -371,14 +477,19 @@ try right_source.addRow({"AAPL", 2, 98}); right_source.addRow({"AAPL", 3, 99}); right_source.addRow({"AMZN", 1, 100}); + right_source.addRow({"AMZN", 2, 0}); + right_source.addChunk(); right_source.addRow({"AMZN", 2, 110}); right_source.addChunk(); right_source.addRow({"AMZN", 4, 130}); right_source.addRow({"AMZN", 5, 140}); + right_source.addRow({"SBUX", 8, 180}); + right_source.addChunk(); + right_source.addRow({"SBUX", 9, 190}); { Block result_block = executePipeline(buildJoinPipeline( - left_source.build(), right_source.build(), /* key_length = */ 2, + left_source.getSource(), right_source.getSource(), /* key_length = */ 2, JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals)); auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); @@ -388,18 +499,19 @@ try })); } - // { - // Block result_block = executePipeline(buildJoinPipeline( - // left_source.build(), right_source.build(), /* key_length = */ 2, - // JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals)); - // auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); + { + Block result_block = executePipeline(buildJoinPipeline( + left_source.getSource(), right_source.getSource(), /* key_length = */ 2, + JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals)); + auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"}); - // ASSERT_EQ(values, (std::vector>{ - // {"AMZN", 3u, 2u, 110u}, - // {"AMZN", 4u, 4u, 130u}, - // {"AMZN", 6u, 5u, 140u}, - // })); - // } + ASSERT_EQ(values, (std::vector>{ + {"AMZN", 3u, 2u, 110u}, + {"AMZN", 4u, 4u, 130u}, + {"AMZN", 6u, 5u, 140u}, + {"SBUX", 10u, 9u, 190u}, + })); + } } catch (Exception & e) { @@ -418,8 +530,7 @@ try {std::make_shared(), "value"}, }); - UInt64 p = std::uniform_int_distribution<>(0, 2)(rng); - double break_prob = p == 0 ? 0.0 : (p == 1 ? 0.5 : 1.0); + double break_prob = getRandomDoubleQuantized(2); std::uniform_real_distribution<> prob_dis(0.0, 1.0); for (const auto & row : std::vector>{ {1, 101}, {2, 102}, {4, 104}, {5, 105}, {11, 111}, {15, 115} }) { @@ -427,7 +538,7 @@ try if (prob_dis(rng) < break_prob) right_source_builder.addChunk(); } - auto right_source = right_source_builder.build(); + auto right_source = right_source_builder.getSource(); auto pipeline = buildJoinPipeline( left_source, right_source, /* key_length = */ 1, @@ -436,7 +547,7 @@ try Block result_block = executePipeline(std::move(pipeline)); ASSERT_EQ( - assert_cast(result_block.getByName("t1.x").column.get())->getData(), + assert_cast(result_block.getByName("t1.key").column.get())->getData(), (ColumnUInt64::Container{3, 3, 3, 3, 3, 5, 5, 6, 9, 9, 10}) ); @@ -456,15 +567,14 @@ catch (Exception & e) throw; } -TEST(FullSortingJoin, AsofGeneratedTestData) +TEST(FullSortingJoin, AsofLessGeneratedTestData) try { - auto join_kind = getRandomFrom({JoinKind::Inner, JoinKind::Left}); + auto join_kind = getRandomFrom({ JoinKind::Inner, JoinKind::Left }); - auto asof_inequality = getRandomFrom({ - ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals, - // ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals, - }); + auto asof_inequality = getRandomFrom({ ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals }); + + SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); SourceChunksBuilder left_source_builder({ {std::make_shared(), "k1"}, @@ -485,11 +595,13 @@ try ColumnInt64::Container expected; - UInt64 k1 = 0; - String k2 = "asdfg"; + UInt64 k1 = 1; + String k2 = ""; auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { + generateNextKey(k1, k2); + Int64 left_t = 0; size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); for (size_t i = 0; i < num_left_rows; ++i) @@ -504,18 +616,10 @@ try auto right_t = left_t; for (size_t j = 0; j < num_matches; ++j) { - int min_step = 1; - if (asof_inequality == ASOFJoinInequality::LessOrEquals || asof_inequality == ASOFJoinInequality::GreaterOrEquals) - min_step = 0; + int min_step = isStrict(asof_inequality) ? 1 : 0; right_t += std::uniform_int_distribution<>(min_step, 3)(rng); - bool is_match = false; - - if (asof_inequality == ASOFJoinInequality::LessOrEquals || asof_inequality == ASOFJoinInequality::Less) - is_match = j == 0; - else if (asof_inequality == ASOFJoinInequality::GreaterOrEquals || asof_inequality == ASOFJoinInequality::Greater) - is_match = j == num_matches - 1; - + bool is_match = j == 0; right_source_builder.addRow({k1, k2, right_t, is_match ? 100 * left_t : -1}); } /// next left_t should be greater than right_t not to match with previous rows @@ -523,7 +627,7 @@ try } /// generate some rows with greater left_t to check that they are not matched - num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); + num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 100)(rng) : 0; for (size_t i = 0; i < num_left_rows; ++i) { left_t += std::uniform_int_distribution<>(1, 10)(rng); @@ -532,12 +636,10 @@ try if (join_kind == JoinKind::Left) expected.push_back(-10 * left_t); } - - generateNextKey(k1, k2); } Block result_block = executePipeline(buildJoinPipeline( - left_source_builder.build(), right_source_builder.build(), + left_source_builder.getSource(), right_source_builder.getSource(), /* key_length = */ 3, join_kind, JoinStrictness::Asof, asof_inequality)); @@ -548,7 +650,97 @@ try assertColumnVectorEq(expected, result_block, "t2.attr"); } -catch (Exception & e) { +catch (Exception & e) +{ + std::cout << e.getStackTraceString() << std::endl; + throw; +} + +TEST(FullSortingJoin, AsofGreaterGeneratedTestData) +try +{ + auto join_kind = getRandomFrom({ JoinKind::Inner, JoinKind::Left }); + + auto asof_inequality = getRandomFrom({ ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals }); + + SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); + + SourceChunksBuilder left_source_builder({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "t"}, + {std::make_shared(), "attr"}, + }); + + SourceChunksBuilder right_source_builder({ + {std::make_shared(), "k1"}, + {std::make_shared(), "k2"}, + {std::make_shared(), "t"}, + {std::make_shared(), "attr"}, + }); + + left_source_builder.break_prob = getRandomDoubleQuantized(); + right_source_builder.break_prob = getRandomDoubleQuantized(); + + ColumnInt64::Container expected; + + UInt64 k1 = 1; + String k2 = ""; + UInt64 left_t = 0; + + auto key_num_total = std::uniform_int_distribution<>(1, 100)(rng); + for (size_t key_num = 0; key_num < key_num_total; ++key_num) + { + generateNextKey(k1, k2); + + /// generate some rows with smaller left_t to check that they are not matched + size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 10)(rng) : 0; + for (size_t i = 0; i < num_left_rows; ++i) + { + left_t += std::uniform_int_distribution<>(1, 10)(rng); + left_source_builder.addRow({k1, k2, left_t, -10 * left_t}); + + if (join_kind == JoinKind::Left) + expected.push_back(-10 * left_t); + } + + if (std::bernoulli_distribution(0.1)(rng)) + continue; + + size_t num_right_matches = std::uniform_int_distribution<>(1, 10)(rng); + auto right_t = left_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 0 : 1, 10)(rng); + for (size_t j = 0; j < num_right_matches; ++j) + { + right_t += std::uniform_int_distribution<>(0, 3)(rng); + bool is_match = j == num_right_matches - 1; + right_source_builder.addRow({k1, k2, right_t, is_match ? 100 * right_t : -1}); + } + + /// next left_t should be greater than (or equals) right_t to match with previous rows + left_t = right_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 1 : 0, 10)(rng); + size_t num_left_matches = std::uniform_int_distribution<>(1, 10)(rng); + for (size_t j = 0; j < num_left_matches; ++j) + { + left_t += std::uniform_int_distribution<>(0, 3)(rng); + left_source_builder.addRow({k1, k2, left_t, 10 * right_t}); + expected.push_back(10 * right_t); + } + } + + Block result_block = executePipeline(buildJoinPipeline( + left_source_builder.getSource(), right_source_builder.getSource(), + /* key_length = */ 3, + join_kind, JoinStrictness::Asof, asof_inequality)); + + assertColumnVectorEq(expected, result_block, "t1.attr"); + + for (auto & e : expected) + e = e < 0 ? 0 : 10 * e; /// non matched rows from left table have negative attr + + assertColumnVectorEq(expected, result_block, "t2.attr"); +} +catch (Exception & e) +{ std::cout << e.getStackTraceString() << std::endl; throw; } diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql index a4e60ff54dd..03936107563 100644 --- a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -19,7 +19,7 @@ SELECT * FROM t1 ANTI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENT SELECT * FROM t1 SEMI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } -SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serverError NOT_IMPLEMENTED } +-- SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED } From b6b55cfb187ddcbb6316d905c60d9be481bfb1a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Sep 2023 09:05:50 +0000 Subject: [PATCH 10/74] fix asof join --- src/Processors/Transforms/MergeJoinTransform.cpp | 2 +- src/Processors/tests/gtest_full_sorting_join.cpp | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 638f2f1cb10..ad564557c36 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1171,7 +1171,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } /// check if blocks are not intersecting at all - if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor, null_direction_hint); cmp != 0) + if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor, null_direction_hint); cmp != 0 && strictness != JoinStrictness::Asof) { if (cmp < 0) { diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 4e0727779b7..2ecf7805df4 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -30,13 +30,13 @@ namespace { [[ maybe_unused ]] -String dumpBlock(std::shared_ptr source) +String dumpBlockSource(std::shared_ptr source, bool mono_block = false) { WriteBufferFromOwnString buf; { Block header = source->getPort().getHeader(); QueryPipeline pipeline(source); - auto format = std::make_shared(buf, header, FormatSettings{}, false); + auto format = std::make_shared(buf, header, FormatSettings{}, mono_block); pipeline.complete(std::move(format)); CompletedPipelineExecutor executor(pipeline); @@ -51,7 +51,7 @@ String dumpBlock(const Block & block) Block header = block.cloneEmpty(); Chunk data(block.getColumns(), block.rows()); auto source = std::make_shared(header, std::move(data)); - return dumpBlock(std::move(source)); + return dumpBlockSource(std::move(source)); } UInt64 getAndPrintRandomSeed() @@ -732,6 +732,10 @@ try /* key_length = */ 3, join_kind, JoinStrictness::Asof, asof_inequality)); + // std::cerr << "============ left ============" << std::endl << dumpBlockSource(left_source_builder.getSource()) << std::endl; + // std::cerr << "============ right ============" << std::endl << dumpBlockSource(right_source_builder.getSource()) << std::endl; + // std::cerr << "============ result ============" << std::endl << dumpBlock(result_block) << std::endl; + assertColumnVectorEq(expected, result_block, "t1.attr"); for (auto & e : expected) From d015a023bb85c72c2747cebed12d065d5aad1159 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Sep 2023 09:56:30 +0000 Subject: [PATCH 11/74] unique rng seed per each test in gtest_full_sorting_join --- .../tests/gtest_full_sorting_join.cpp | 122 ++++++++++-------- 1 file changed, 69 insertions(+), 53 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 2ecf7805df4..2bd3357eff9 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -54,20 +54,6 @@ String dumpBlock(const Block & block) return dumpBlockSource(std::move(source)); } -UInt64 getAndPrintRandomSeed() -{ - UInt64 seed = randomSeed(); - if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe) - seed = std::stoull(random_seed); - - std::cerr << __FILE__ << " :: " << "TEST_RANDOM_SEED=" << seed << std::endl; - return seed; -} - -static UInt64 TEST_RANDOM_SEED = getAndPrintRandomSeed(); -static pcg64 rng(TEST_RANDOM_SEED); - - QueryPipeline buildJoinPipeline( std::shared_ptr left_source, std::shared_ptr right_source, @@ -154,7 +140,6 @@ std::shared_ptr oneColumnSource(const std::vector> class SourceChunksBuilder { public: - double break_prob = 0.0; explicit SourceChunksBuilder(const Block & header_) : header(header_) @@ -163,13 +148,20 @@ public: chassert(!current_chunk.empty()); } + void setBreakProbability(pcg64 & rng_) + { + /// random probability with possibility to have exact 0.0 and 1.0 values + break_prob = std::uniform_int_distribution(0, 5)(rng_) / static_cast(5); + rng = &rng_; + } + void addRow(const std::vector & row) { chassert(row.size() == current_chunk.size()); for (size_t i = 0; i < current_chunk.size(); ++i) current_chunk[i]->insert(row[i]); - if (break_prob > 0.0 && std::uniform_real_distribution<>(0.0, 1.0)(rng) < break_prob) + if (rng && std::uniform_real_distribution<>(0.0, 1.0)(*rng) < break_prob) addChunk(); } @@ -200,6 +192,9 @@ private: Block header; Chunks chunks; MutableColumns current_chunk; + + pcg64 * rng = nullptr; + double break_prob = 0.0; }; @@ -239,8 +234,20 @@ void assertColumnVectorEq(const typename ColumnVector::Container & expected, const auto * actual = typeid_cast *>(block.getByName(name).column.get()); ASSERT_TRUE(actual) << "unexpected column type: " << block.getByName(name).column->dumpStructure() << "expected: " << typeid(ColumnVector).name(); + auto get_first_diff = [&]() -> String + { + const auto & actual_data = actual->getData(); + size_t num_rows = std::min(expected.size(), actual_data.size()); + for (size_t i = 0; i < num_rows; ++i) + { + if (expected[i] != actual_data[i]) + return fmt::format(", expected: {}, actual: {} at row {}", expected[i], actual_data[i], i); + } + return ""; + }; + EXPECT_EQ(actual->getData().size(), expected.size()); - ASSERT_EQ(actual->getData(), expected) << "column name: " << name; + ASSERT_EQ(actual->getData(), expected) << "column name: " << name << get_first_diff(); } template @@ -264,20 +271,14 @@ void assertColumnEq(const IColumn & expected, const Block & block, const std::st } template -T getRandomFrom(const std::initializer_list & opts) +T getRandomFrom(pcg64 & rng, const std::initializer_list & opts) { std::vector options(opts.begin(), opts.end()); size_t idx = std::uniform_int_distribution(0, options.size() - 1)(rng); return options[idx]; } -/// Used to have accurate 0.0 and 1.0 probabilities -double getRandomDoubleQuantized(size_t quants = 5) -{ - return std::uniform_int_distribution(0, quants)(rng) / static_cast(quants); -} - -void generateNextKey(UInt64 & k1, String & k2) +void generateNextKey(pcg64 & rng, UInt64 & k1, String & k2) { size_t str_len = std::uniform_int_distribution<>(1, 10)(rng); String new_k2 = getRandomASCIIString(str_len, rng); @@ -293,6 +294,28 @@ bool isStrict(ASOFJoinInequality inequality) } +class FullSortingJoinRandomized : public ::testing::Test +{ +public: + FullSortingJoinRandomized() = default; + + void SetUp() override + { + UInt64 seed = randomSeed(); + if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe) + seed = std::stoull(random_seed); + + std::cerr << "TEST_RANDOM_SEED=" << seed << std::endl; + rng = pcg64(seed); + } + + void TearDown() override + { + } + + pcg64 rng; +}; + TEST(FullSortingJoin, AllAnyOneKey) try { @@ -369,10 +392,10 @@ catch (Exception & e) } -TEST(FullSortingJoin, AnyRandomized) +TEST_F(FullSortingJoinRandomized, Any) try { - JoinKind kind = getRandomFrom({JoinKind::Inner, JoinKind::Left, JoinKind::Right}); + JoinKind kind = getRandomFrom(rng, {JoinKind::Inner, JoinKind::Left, JoinKind::Right}); SourceChunksBuilder left_source({ {std::make_shared(), "k1"}, @@ -386,8 +409,8 @@ try {std::make_shared(), "attr"}, }); - left_source.break_prob = getRandomDoubleQuantized(); - right_source.break_prob = getRandomDoubleQuantized(); + left_source.setBreakProbability(rng); + right_source.setBreakProbability(rng); size_t num_keys = std::uniform_int_distribution<>(100, 1000)(rng); @@ -404,7 +427,7 @@ try for (size_t i = 0; i < num_keys; ++i) { - generateNextKey(k1, k2); + generateNextKey(rng, k1, k2); /// Key is present in left, right or both tables. Both tables is more probable. size_t key_presence = std::uniform_int_distribution<>(0, 10)(rng); @@ -520,7 +543,7 @@ catch (Exception & e) } -TEST(FullSortingJoin, AsofOnlyColumn) +TEST_F(FullSortingJoinRandomized, AsofOnlyColumn) try { auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} }); @@ -530,14 +553,11 @@ try {std::make_shared(), "value"}, }); - double break_prob = getRandomDoubleQuantized(2); - std::uniform_real_distribution<> prob_dis(0.0, 1.0); + right_source_builder.setBreakProbability(rng); + for (const auto & row : std::vector>{ {1, 101}, {2, 102}, {4, 104}, {5, 105}, {11, 111}, {15, 115} }) - { right_source_builder.addRow(row); - if (prob_dis(rng) < break_prob) - right_source_builder.addChunk(); - } + auto right_source = right_source_builder.getSource(); auto pipeline = buildJoinPipeline( @@ -567,12 +587,12 @@ catch (Exception & e) throw; } -TEST(FullSortingJoin, AsofLessGeneratedTestData) +TEST_F(FullSortingJoinRandomized, AsofLessGeneratedTestData) try { - auto join_kind = getRandomFrom({ JoinKind::Inner, JoinKind::Left }); + auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); - auto asof_inequality = getRandomFrom({ ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals }); + auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals }); SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); @@ -590,8 +610,8 @@ try {std::make_shared(), "attr"}, }); - left_source_builder.break_prob = getRandomDoubleQuantized(); - right_source_builder.break_prob = getRandomDoubleQuantized(); + left_source_builder.setBreakProbability(rng); + right_source_builder.setBreakProbability(rng); ColumnInt64::Container expected; @@ -600,7 +620,7 @@ try auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { - generateNextKey(k1, k2); + generateNextKey(rng, k1, k2); Int64 left_t = 0; size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); @@ -656,12 +676,12 @@ catch (Exception & e) throw; } -TEST(FullSortingJoin, AsofGreaterGeneratedTestData) +TEST_F(FullSortingJoinRandomized, AsofGreaterGeneratedTestData) try { - auto join_kind = getRandomFrom({ JoinKind::Inner, JoinKind::Left }); + auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); - auto asof_inequality = getRandomFrom({ ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals }); + auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals }); SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); @@ -679,8 +699,8 @@ try {std::make_shared(), "attr"}, }); - left_source_builder.break_prob = getRandomDoubleQuantized(); - right_source_builder.break_prob = getRandomDoubleQuantized(); + left_source_builder.setBreakProbability(rng); + right_source_builder.setBreakProbability(rng); ColumnInt64::Container expected; @@ -691,7 +711,7 @@ try auto key_num_total = std::uniform_int_distribution<>(1, 100)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { - generateNextKey(k1, k2); + generateNextKey(rng, k1, k2); /// generate some rows with smaller left_t to check that they are not matched size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 10)(rng) : 0; @@ -732,10 +752,6 @@ try /* key_length = */ 3, join_kind, JoinStrictness::Asof, asof_inequality)); - // std::cerr << "============ left ============" << std::endl << dumpBlockSource(left_source_builder.getSource()) << std::endl; - // std::cerr << "============ right ============" << std::endl << dumpBlockSource(right_source_builder.getSource()) << std::endl; - // std::cerr << "============ result ============" << std::endl << dumpBlock(result_block) << std::endl; - assertColumnVectorEq(expected, result_block, "t1.attr"); for (auto & e : expected) From 3e9090cbebb3cdb97f4dbb4b9d03c7e9d9e242b1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Sep 2023 11:14:37 +0000 Subject: [PATCH 12/74] fix --- .../sql-reference/statements/select/join.md | 5 ++- .../Transforms/MergeJoinTransform.cpp | 41 ++++++++++++------- .../Transforms/MergeJoinTransform.h | 23 +---------- .../tests/gtest_full_sorting_join.cpp | 36 +++++++++------- 4 files changed, 52 insertions(+), 53 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 34c6016235a..96d9d26977d 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -297,7 +297,7 @@ Algorithm requires the special column in tables. This column: - Must contain an ordered sequence. - Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). -- Can’t be the only column in the `JOIN` clause. +- For `hash` join algorithm it can’t be the only column in the `JOIN` clause. Syntax `ASOF JOIN ... ON`: @@ -337,7 +337,8 @@ For example, consider the following tables: `ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. :::note -`ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. +`ASOF JOIN` is supported only by `hash` and `full_sorting_merge` join algorithms. +It's **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. ::: ## PASTE JOIN Usage diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index ad564557c36..0d9eab248d8 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -329,6 +329,26 @@ void AsofJoinState::reset() value.clear(); } +FullMergeJoinCursor::FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_, bool is_asof) + : sample_block(sample_block_.cloneEmpty()) + , desc(description_) +{ + if (desc.size() == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty sort description for FullMergeJoinCursor"); + + if (is_asof) + { + /// For ASOF join prefix of sort description is used for equality comparison + /// and the last column is used for inequality comparison and is handled separately + + auto asof_column_description = desc.back(); + desc.pop_back(); + + chassert(asof_column_description.direction == 1 && asof_column_description.nulls_direction == 1); + asof_column_position = sample_block.getPositionByName(asof_column_description.column_name); + } +} + const Chunk & FullMergeJoinCursor::getCurrent() const { return current_chunk; @@ -686,7 +706,6 @@ std::optional MergeJoinAlgorithm::handleAsofJoinStat return {}; auto & left_cursor = *cursors[0]; - size_t lpos = left_cursor->getRow(); const auto & left_columns = left_cursor.getCurrent().getColumns(); MutableColumns result_cols = getEmptyResultColumns(); @@ -695,7 +714,7 @@ std::optional MergeJoinAlgorithm::handleAsofJoinStat { size_t i = 0; for (const auto & col : left_columns) - result_cols[i++]->insertFrom(*col, lpos); + result_cols[i++]->insertFrom(*col, left_cursor->getRow()); for (const auto & col : asof_join_state.value.getColumns()) result_cols[i++]->insertFrom(*col, asof_join_state.value_row); chassert(i == result_cols.size()); @@ -707,7 +726,7 @@ std::optional MergeJoinAlgorithm::handleAsofJoinStat /// return row with default values at right side size_t i = 0; for (const auto & col : left_columns) - result_cols[i++]->insertFrom(*col, lpos); + result_cols[i++]->insertFrom(*col, left_cursor->getRow()); for (; i < result_cols.size(); ++i) result_cols[i]->insertDefault(); chassert(i == result_cols.size()); @@ -977,12 +996,10 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() auto lpos = left_cursor->getRow(); auto rpos = right_cursor->getRow(); auto cmp = compareCursors(*left_cursor, *right_cursor); - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ({}) <=> ({}) -> {}", __FILE__, __LINE__, left_cursor.dump(), right_cursor.dump(), cmp); if (cmp == 0) { auto asof_cmp = compareAsofCursors(left_cursor, right_cursor); - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ({}) <=> ({}) -> asof {}", __FILE__, __LINE__, left_cursor.dump(), right_cursor.dump(), asof_cmp); if ((asof_inequality == ASOFJoinInequality::Less && asof_cmp <= -1) || (asof_inequality == ASOFJoinInequality::LessOrEquals && asof_cmp <= 0)) @@ -1010,7 +1027,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() if ((asof_inequality == ASOFJoinInequality::Greater && asof_cmp >= 1) || (asof_inequality == ASOFJoinInequality::GreaterOrEquals && asof_cmp >= 0)) { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); /// condition is satisfied, remember this row and move next to try to find better match asof_join_state.set(right_cursor, rpos); right_cursor->next(); @@ -1022,7 +1038,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() /// Asof condition is not satisfied anymore, use last matched row from right table if (asof_join_state.hasMatch(left_cursor, asof_inequality)) { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); size_t i = 0; for (const auto & col : left_columns) result_cols[i++]->insertFrom(*col, lpos); @@ -1035,7 +1050,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() asof_join_state.reset(); if (isLeft(kind)) { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); /// return row with default values at right side size_t i = 0; @@ -1047,7 +1061,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() } } left_cursor->next(); - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); continue; } @@ -1055,10 +1068,8 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() } else if (cmp < 0) { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); if (asof_join_state.hasMatch(left_cursor, asof_inequality)) { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); size_t i = 0; for (const auto & col : left_columns) @@ -1071,13 +1082,12 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() } else { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); asof_join_state.reset(); } - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); /// no matches for rows in left table, just pass them through size_t num = nextDistinct(*left_cursor); + if (isLeft(kind) && num) { /// return them with default values at right side @@ -1091,7 +1101,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() } else { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); /// skip rows in right table until we find match for current row in left table nextDistinct(*right_cursor); @@ -1106,6 +1115,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() /// otherwise - vice versa Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const { + ColumnRawPtrs cols; { const auto & columns_left = source_num == 0 ? cursors[0]->getCurrent().getColumns() : cursors[0]->sampleColumns(); @@ -1128,7 +1138,6 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t star cols.push_back(col.get()); } } - Chunk result_chunk; copyColumnsResized(cols, start, num_rows, result_chunk); return result_chunk; @@ -1144,6 +1153,7 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { + if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) return Status(0); @@ -1161,6 +1171,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { + if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) return Status(createBlockWithDefaults(0)); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index dbdda0b166b..c8ba857781e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -185,28 +185,7 @@ public: class FullMergeJoinCursor : boost::noncopyable { public: - FullMergeJoinCursor( - const Block & sample_block_, - const SortDescription & description_, - bool is_asof = false) - : sample_block(sample_block_.cloneEmpty()) - , desc(description_) - { - if (desc.size() == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty sort description for FullMergeJoinCursor"); - - if (is_asof) - { - /// For ASOF join prefix of sort description is used for equality comparison - /// and the last column is used for inequality comparison and is handled separately - - auto asof_column_description = desc.back(); - desc.pop_back(); - - chassert(asof_column_description.direction == 1 && asof_column_description.nulls_direction == 1); - asof_column_position = sample_block.getPositionByName(asof_column_description.column_name); - } - } + FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_, bool is_asof = false); bool fullyCompleted() const; void setChunk(Chunk && chunk); diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 2bd3357eff9..e3423aa0386 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -294,18 +295,25 @@ bool isStrict(ASOFJoinInequality inequality) } -class FullSortingJoinRandomized : public ::testing::Test +class FullSortingJoinTest : public ::testing::Test { public: - FullSortingJoinRandomized() = default; + FullSortingJoinTest() = default; void SetUp() override { + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + if (const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe) + Poco::Logger::root().setLevel(test_log_level); + else + Poco::Logger::root().setLevel("none"); + + UInt64 seed = randomSeed(); if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe) seed = std::stoull(random_seed); - - std::cerr << "TEST_RANDOM_SEED=" << seed << std::endl; + std::cout << "TEST_RANDOM_SEED=" << seed << std::endl; rng = pcg64(seed); } @@ -316,7 +324,7 @@ public: pcg64 rng; }; -TEST(FullSortingJoin, AllAnyOneKey) +TEST_F(FullSortingJoinTest, AllAnyOneKey) try { { @@ -392,7 +400,7 @@ catch (Exception & e) } -TEST_F(FullSortingJoinRandomized, Any) +TEST_F(FullSortingJoinTest, AnySimple) try { JoinKind kind = getRandomFrom(rng, {JoinKind::Inner, JoinKind::Left, JoinKind::Right}); @@ -478,7 +486,7 @@ catch (Exception & e) throw; } -TEST(FullSortingJoin, Asof) +TEST_F(FullSortingJoinTest, AsofSimple) try { SourceChunksBuilder left_source({ @@ -543,7 +551,7 @@ catch (Exception & e) } -TEST_F(FullSortingJoinRandomized, AsofOnlyColumn) +TEST_F(FullSortingJoinTest, AsofOnlyColumn) try { auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} }); @@ -587,7 +595,7 @@ catch (Exception & e) throw; } -TEST_F(FullSortingJoinRandomized, AsofLessGeneratedTestData) +TEST_F(FullSortingJoinTest, AsofLessGeneratedTestData) try { auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); @@ -676,7 +684,7 @@ catch (Exception & e) throw; } -TEST_F(FullSortingJoinRandomized, AsofGreaterGeneratedTestData) +TEST_F(FullSortingJoinTest, AsofGreaterGeneratedTestData) try { auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); @@ -708,13 +716,13 @@ try String k2 = ""; UInt64 left_t = 0; - auto key_num_total = std::uniform_int_distribution<>(1, 100)(rng); + auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { generateNextKey(rng, k1, k2); /// generate some rows with smaller left_t to check that they are not matched - size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 10)(rng) : 0; + size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 100)(rng) : 0; for (size_t i = 0; i < num_left_rows; ++i) { left_t += std::uniform_int_distribution<>(1, 10)(rng); @@ -737,8 +745,8 @@ try } /// next left_t should be greater than (or equals) right_t to match with previous rows - left_t = right_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 1 : 0, 10)(rng); - size_t num_left_matches = std::uniform_int_distribution<>(1, 10)(rng); + left_t = right_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 1 : 0, 100)(rng); + size_t num_left_matches = std::uniform_int_distribution<>(1, 100)(rng); for (size_t j = 0; j < num_left_matches; ++j) { left_t += std::uniform_int_distribution<>(0, 3)(rng); From a8690947c705339113266c1ec8db05695a36e472 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 28 Sep 2023 12:50:24 +0000 Subject: [PATCH 13/74] enable stateless tessts for full sorting asof join --- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++- src/Planner/PlannerJoinTree.cpp | 6 ++- .../Transforms/MergeJoinTransform.cpp | 2 - .../00927_asof_join_correct_bt.reference | 15 +++++++ .../00927_asof_join_correct_bt.sql | 30 ++++++++----- .../00927_asof_join_long.reference | 1 + .../0_stateless/00927_asof_join_long.sql | 17 ++++++- .../00927_asof_join_noninclusive.reference | 29 ++++++++++++ .../00927_asof_join_noninclusive.sql | 5 ++- .../00927_asof_join_other_types.reference | 45 +++++++++++++++++++ .../00927_asof_join_other_types.sh | 27 ----------- .../00927_asof_join_other_types.sql.j2 | 27 +++++++++++ .../0_stateless/00927_asof_joins.reference | 15 +++++++ .../queries/0_stateless/00927_asof_joins.sql | 8 +++- .../0_stateless/00976_asof_join_on.reference | 38 ++++++++++++++++ ..._join_on.sql => 00976_asof_join_on.sql.j2} | 8 ++++ 16 files changed, 231 insertions(+), 49 deletions(-) delete mode 100755 tests/queries/0_stateless/00927_asof_join_other_types.sh create mode 100755 tests/queries/0_stateless/00927_asof_join_other_types.sql.j2 rename tests/queries/0_stateless/{00976_asof_join_on.sql => 00976_asof_join_on.sql.j2} (90%) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b72399df2c1..a0b53ad42d1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1726,7 +1726,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getCurrentDataStream().header, join_clause.key_names_right); - if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering && has_non_const_keys) + if (settings.max_rows_in_set_to_optimize_join > 0 && join_type_allows_filtering && has_non_const_keys) { auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index aa46f65d2d4..63acf194139 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1484,7 +1484,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ { const auto & join_clause = table_join->getOnlyClause(); - bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); + bool join_type_allows_filtering = (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Any) + && (isInner(join_kind) || isLeft(join_kind) || isRight(join_kind)); + auto has_non_const = [](const Block & block, const auto & keys) { @@ -1504,7 +1506,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ bool has_non_const_keys = has_non_const(left_plan.getCurrentDataStream().header, join_clause.key_names_left) && has_non_const(right_plan.getCurrentDataStream().header, join_clause.key_names_right); - if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering && has_non_const_keys) + if (settings.max_rows_in_set_to_optimize_join > 0 && join_type_allows_filtering && has_non_const_keys) { auto * left_set = add_create_set(left_plan, join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(right_plan, join_clause.key_names_right, JoinTableSide::Right); diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 0d9eab248d8..df56ffc2871 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -456,8 +456,6 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( size_t left_idx = input_headers[0].getPositionByName(left_key); size_t right_idx = input_headers[1].getPositionByName(right_key); left_to_right_key_remap[left_idx] = right_idx; - if (strictness == JoinStrictness::Asof) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ASOF joins USING"); } const auto *smjPtr = typeid_cast(table_join.get()); diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.reference b/tests/queries/0_stateless/00927_asof_join_correct_bt.reference index bb199d0159a..a398f9604fd 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.reference +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.reference @@ -13,3 +13,18 @@ 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +1 101 1 0 0 0 +1 102 2 2 102 1 +1 103 3 2 102 1 +1 104 4 4 104 1 +1 105 5 4 104 1 +1 101 1 0 0 0 +1 102 2 2 102 1 +1 103 3 2 102 1 +1 104 4 4 104 1 +1 105 5 4 104 1 +1 101 1 0 0 0 +1 102 2 2 102 1 +1 103 3 2 102 1 +1 104 4 4 104 1 +1 105 5 4 104 1 diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql index 281a81d51c0..761d6bacde6 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql @@ -4,20 +4,26 @@ DROP TABLE IF EXISTS B; CREATE TABLE A(k UInt32, t UInt32, a UInt64) ENGINE = MergeTree() ORDER BY (k, t); INSERT INTO A(k,t,a) VALUES (1,101,1),(1,102,2),(1,103,3),(1,104,4),(1,105,5); -CREATE TABLE B(k UInt32, t UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); -INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); -SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); -DROP TABLE B; +CREATE TABLE B1(k UInt32, t UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); +INSERT INTO B1(k,t,b) VALUES (1,102,2), (1,104,4); +CREATE TABLE B2(t UInt32, k UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); +INSERT INTO B2(k,t,b) VALUES (1,102,2), (1,104,4); -CREATE TABLE B(t UInt32, k UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); -INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); -SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); -DROP TABLE B; +CREATE TABLE B3(k UInt32, b UInt64, t UInt32) ENGINE = MergeTree() ORDER BY (k, t); +INSERT INTO B3(k,t,b) VALUES (1,102,2), (1,104,4); -CREATE TABLE B(k UInt32, b UInt64, t UInt32) ENGINE = MergeTree() ORDER BY (k, t); -INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); -SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); -DROP TABLE B; +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t); +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); + +SET join_algorithm = 'full_sorting_merge'; +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t); +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); + +DROP TABLE B1; +DROP TABLE B2; +DROP TABLE B3; DROP TABLE A; diff --git a/tests/queries/0_stateless/00927_asof_join_long.reference b/tests/queries/0_stateless/00927_asof_join_long.reference index d4f015c68e4..ec40d2bc463 100644 --- a/tests/queries/0_stateless/00927_asof_join_long.reference +++ b/tests/queries/0_stateless/00927_asof_join_long.reference @@ -1 +1,2 @@ 3000000 +3000000 diff --git a/tests/queries/0_stateless/00927_asof_join_long.sql b/tests/queries/0_stateless/00927_asof_join_long.sql index c03a06d48d4..7a73875e93e 100644 --- a/tests/queries/0_stateless/00927_asof_join_long.sql +++ b/tests/queries/0_stateless/00927_asof_join_long.sql @@ -2,15 +2,28 @@ DROP TABLE IF EXISTS tvs; +-- to use different algorithms for in subquery +SET allow_experimental_analyzer = 1; + CREATE TABLE tvs(k UInt32, t UInt32, tv UInt64) ENGINE = Memory; INSERT INTO tvs(k,t,tv) SELECT k, t, t FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys -CROSS JOIN (SELECT toUInt32(number * 3) as t FROM numbers(10000)) tv_times; +CROSS JOIN (SELECT toUInt32(number * 3) as t FROM numbers(10000)) tv_times +SETTINGS join_algorithm = 'hash'; SELECT SUM(trades.price - tvs.tv) FROM (SELECT k, t, t as price FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys - CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times) trades + CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times + SETTINGS join_algorithm = 'hash') trades ASOF LEFT JOIN tvs USING(k,t); +SELECT SUM(trades.price - tvs.tv) FROM +(SELECT k, t, t as price + FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys + CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times + SETTINGS join_algorithm = 'hash') trades +ASOF LEFT JOIN tvs USING(k,t) +SETTINGS join_algorithm = 'full_sorting_merge'; + DROP TABLE tvs; diff --git a/tests/queries/0_stateless/00927_asof_join_noninclusive.reference b/tests/queries/0_stateless/00927_asof_join_noninclusive.reference index fe2844a2a43..d856372fb4a 100644 --- a/tests/queries/0_stateless/00927_asof_join_noninclusive.reference +++ b/tests/queries/0_stateless/00927_asof_join_noninclusive.reference @@ -27,3 +27,32 @@ 2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2 2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2 2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2 +1 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0 +1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1 +1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1 +2 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0 +2 1970-01-01 00:00:02 2 0 1970-01-01 00:00:00 0 +2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2 +3 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0 +3 1970-01-01 00:00:02 2 0 1970-01-01 00:00:00 0 +3 1970-01-01 00:00:03 3 0 1970-01-01 00:00:00 0 +3 1970-01-01 00:00:04 4 0 1970-01-01 00:00:00 0 +3 1970-01-01 00:00:05 5 0 1970-01-01 00:00:00 0 +1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1 +1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1 +2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2 +1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1 +1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1 +1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1 +2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2 +2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2 diff --git a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql index 5f15f3b593d..3cc99df4462 100644 --- a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql +++ b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql @@ -11,9 +11,12 @@ INSERT INTO B(k,t,b) VALUES (1,2,2),(1,4,4); INSERT INTO B(k,t,b) VALUES (2,3,3); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); - SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t); +SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t); +SET join_algorithm = 'full_sorting_merge'; +SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); +SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t); DROP TABLE A; diff --git a/tests/queries/0_stateless/00927_asof_join_other_types.reference b/tests/queries/0_stateless/00927_asof_join_other_types.reference index 80c85ec1ae3..ddbc24ff925 100644 --- a/tests/queries/0_stateless/00927_asof_join_other_types.reference +++ b/tests/queries/0_stateless/00927_asof_join_other_types.reference @@ -1,27 +1,72 @@ +- 2 1 1 0 2 3 3 3 2 5 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- 2 1970-01-01 02:00:01 1 0 2 1970-01-01 02:00:03 3 3 2 1970-01-01 02:00:05 5 3 +- +2 1970-01-01 02:00:01 1 0 +2 1970-01-01 02:00:03 3 3 +2 1970-01-01 02:00:05 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- 2 1 1 0 2 3 3 3 2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1 1 0 +2 3 3 3 +2 5 5 3 +- +2 1970-01-01 02:00:00.001 1 0 +2 1970-01-01 02:00:00.003 3 3 +2 1970-01-01 02:00:00.005 5 3 +- 2 1970-01-01 02:00:00.001 1 0 2 1970-01-01 02:00:00.003 3 3 2 1970-01-01 02:00:00.005 5 3 diff --git a/tests/queries/0_stateless/00927_asof_join_other_types.sh b/tests/queries/0_stateless/00927_asof_join_other_types.sh deleted file mode 100755 index 10173a3e43f..00000000000 --- a/tests/queries/0_stateless/00927_asof_join_other_types.sh +++ /dev/null @@ -1,27 +0,0 @@ -#!/usr/bin/env bash - -set -e - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -for typename in "UInt32" "UInt64" "Float64" "Float32" "DateTime('Asia/Istanbul')" "Decimal32(5)" "Decimal64(5)" "Decimal128(5)" "DateTime64(3, 'Asia/Istanbul')" -do - $CLICKHOUSE_CLIENT -mn <= B.t ORDER BY (A.a, A.t); SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t; SELECT A.a, A.t, B.b, B.t FROM A ASOF INNER JOIN B ON B.t <= A.t AND A.a == B.b ORDER BY (A.a, A.t); @@ -28,5 +34,7 @@ ASOF INNER JOIN (SELECT * FROM B UNION ALL SELECT 1, 3) AS B ON B.t <= A.t AND A WHERE B.t != 3 ORDER BY (A.a, A.t) ; +{% endfor %} + DROP TABLE A; DROP TABLE B; From 256ad60115ddd1eb8c8a5597478e057926fe30f6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 28 Sep 2023 12:52:20 +0000 Subject: [PATCH 14/74] fix style, clang tidy --- src/Planner/PlannerJoinTree.cpp | 8 +-- .../Transforms/MergeJoinTransform.cpp | 59 +++++++++------- .../Transforms/MergeJoinTransform.h | 2 +- .../tests/gtest_full_sorting_join.cpp | 27 +------- .../00927_asof_join_other_types.sql.j2 | 0 .../01116_asof_join_dolbyzerr.reference | 3 + .../0_stateless/01116_asof_join_dolbyzerr.sql | 14 ++++ .../02240_asof_join_biginteger.reference | 4 ++ .../02240_asof_join_biginteger.sql | 8 +++ ...n_in_left_table_clause_asof_join.reference | 1 + ...unction_in_left_table_clause_asof_join.sql | 12 ++++ .../03143_asof_join_ddb_long.reference | 2 + .../0_stateless/03143_asof_join_ddb_long.sql | 48 +++++++++++++ .../03144_asof_join_ddb_doubles.reference | 58 ++++++++++++++++ .../03144_asof_join_ddb_doubles.sql | 64 +++++++++++++++++ ...03145_asof_join_ddb_inequalities.reference | 68 +++++++++++++++++++ .../03145_asof_join_ddb_inequalities.sql | 63 +++++++++++++++++ .../03146_asof_join_ddb_merge_long.reference | 2 + .../03146_asof_join_ddb_merge_long.sql.j2 | 37 ++++++++++ 19 files changed, 424 insertions(+), 56 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/00927_asof_join_other_types.sql.j2 create mode 100644 tests/queries/0_stateless/03143_asof_join_ddb_long.reference create mode 100644 tests/queries/0_stateless/03143_asof_join_ddb_long.sql create mode 100644 tests/queries/0_stateless/03144_asof_join_ddb_doubles.reference create mode 100644 tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql create mode 100644 tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference create mode 100644 tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql create mode 100644 tests/queries/0_stateless/03146_asof_join_ddb_merge_long.reference create mode 100644 tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 63acf194139..a9d9e11f458 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -75,7 +75,6 @@ namespace ErrorCodes extern const int INVALID_JOIN_ON_EXPRESSION; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; - extern const int SYNTAX_ERROR; extern const int ACCESS_DENIED; extern const int PARAMETER_OUT_OF_BOUND; extern const int TOO_MANY_COLUMNS; @@ -1357,12 +1356,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ { if (!join_clause.hasASOF()) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} no inequality in ASOF JOIN ON section.", - join_node.formatASTForErrorMessage()); - - if (table_join_clause.key_names_left.size() <= 1) - throw Exception(ErrorCodes::SYNTAX_ERROR, - "JOIN {} ASOF join needs at least one equi-join column", + "JOIN {} no inequality in ASOF JOIN ON section", join_node.formatASTForErrorMessage()); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index df56ffc2871..26e1ebb0b60 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -43,6 +43,13 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, return std::make_unique(materializeBlock(block), desc, strictness == JoinStrictness::Asof); } +bool isNullAt(const IColumn & column, size_t row) +{ + if (const auto * nullable = checkAndGetColumn(column)) + return nullable->isNullAt(row); + return false; +} + template int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint) { @@ -54,7 +61,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if (left_nullable && right_nullable) { int res = left_nullable->compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); - if (res) + if (res != 0) return res; /// NULL != NULL case @@ -108,9 +115,9 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImp return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), lhs.sort_columns_size, null_direction_hint); } -int compareAsofCursors(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +int compareAsofCursors(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs, int null_direction_hint) { - return nullableCompareAt(*lhs.getAsofColumn(), *rhs.getAsofColumn(), lhs->getRow(), rhs->getRow()); + return nullableCompareAt(*lhs.getAsofColumn(), *rhs.getAsofColumn(), lhs->getRow(), rhs->getRow(), null_direction_hint); } bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) @@ -235,12 +242,6 @@ void inline addRange(PaddedPODArray & values, UInt64 start, UInt64 end) values.push_back(i); } -void inline addMany(PaddedPODArray & left_or_right_map, size_t idx, size_t num) -{ - for (size_t i = 0; i < num; ++i) - left_or_right_map.push_back(idx); -} - void inline addMany(PaddedPODArray & values, UInt64 value, size_t num) { values.resize_fill(values.size() + num, value); @@ -257,7 +258,7 @@ JoinKeyRow::JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos) new_col->insertFrom(*col, pos); row.push_back(std::move(new_col)); } - if (auto asof_column = cursor.getAsofColumn()) + if (const auto * asof_column = cursor.getAsofColumn()) { auto new_col = asof_column->cloneEmpty(); new_col->insertFrom(*asof_column, pos); @@ -275,10 +276,10 @@ bool JoinKeyRow::equals(const FullMergeJoinCursor & cursor) const if (row.empty()) return false; - assert(this->row.size() == cursor->sort_columns_size); for (size_t i = 0; i < cursor->sort_columns_size; ++i) { - int cmp = this->row[i]->compareAt(0, cursor->getRow(), *(cursor->sort_columns[i]), cursor->desc[i].nulls_direction); + // int cmp = this->row[i]->compareAt(0, cursor->getRow(), *(cursor->sort_columns[i]), cursor->desc[i].nulls_direction); + int cmp = nullableCompareAt(*this->row[i], *cursor->sort_columns[i], 0, cursor->getRow(), cursor->desc[i].nulls_direction); if (cmp != 0) return false; } @@ -287,9 +288,16 @@ bool JoinKeyRow::equals(const FullMergeJoinCursor & cursor) const bool JoinKeyRow::asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const { + chassert(this->row.size() == cursor->sort_columns_size + 1); if (!equals(cursor)) return false; - int cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *row.back(), 1); + + const auto & asof_row = row.back(); + if (isNullAt(*asof_row, 0) || isNullAt(*cursor.getAsofColumn(), cursor->getRow())) + return false; + + int cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *asof_row, 1); + return (asof_inequality == ASOFJoinInequality::Less && cmp < 0) || (asof_inequality == ASOFJoinInequality::LessOrEquals && cmp <= 0) || (asof_inequality == ASOFJoinInequality::Greater && cmp > 0) @@ -333,7 +341,7 @@ FullMergeJoinCursor::FullMergeJoinCursor(const Block & sample_block_, const Sort : sample_block(sample_block_.cloneEmpty()) , desc(description_) { - if (desc.size() == 0) + if (desc.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty sort description for FullMergeJoinCursor"); if (is_asof) @@ -393,7 +401,7 @@ String FullMergeJoinCursor::dump() const row_dump.push_back(val.dump()); } - if (auto * asof_column = getAsofColumn()) + if (const auto * asof_column = getAsofColumn()) { asof_column->get(cursor.getRow(), val); row_dump.push_back(val.dump()); @@ -436,9 +444,10 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); cursors = { - createCursor(input_headers[0], on_clause_.key_names_left, strictness), - createCursor(input_headers[1], on_clause_.key_names_right, strictness), + createCursor(input_headers[0], on_clause_.key_names_left, strictness), + createCursor(input_headers[1], on_clause_.key_names_right, strictness), }; +} MergeJoinAlgorithm::MergeJoinAlgorithm( JoinPtr join_ptr, @@ -458,7 +467,7 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( left_to_right_key_remap[left_idx] = right_idx; } - const auto *smjPtr = typeid_cast(table_join.get()); + const auto *smjPtr = typeid_cast(join_ptr.get()); if (smjPtr) { null_direction_hint = smjPtr->getNullDirection(); @@ -993,11 +1002,18 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() { auto lpos = left_cursor->getRow(); auto rpos = right_cursor->getRow(); - auto cmp = compareCursors(*left_cursor, *right_cursor); + auto cmp = compareCursors(*left_cursor, *right_cursor, null_direction_hint); + if (cmp == 0) + { + if (isNullAt(*left_cursor.getAsofColumn(), lpos)) + cmp = -1; + if (isNullAt(*right_cursor.getAsofColumn(), rpos)) + cmp = 1; + } if (cmp == 0) { - auto asof_cmp = compareAsofCursors(left_cursor, right_cursor); + auto asof_cmp = compareAsofCursors(left_cursor, right_cursor, null_direction_hint); if ((asof_inequality == ASOFJoinInequality::Less && asof_cmp <= -1) || (asof_inequality == ASOFJoinInequality::LessOrEquals && asof_cmp <= 0)) @@ -1048,7 +1064,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() asof_join_state.reset(); if (isLeft(kind)) { - /// return row with default values at right side size_t i = 0; for (const auto & col : left_columns) @@ -1068,7 +1083,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() { if (asof_join_state.hasMatch(left_cursor, asof_inequality)) { - size_t i = 0; for (const auto & col : left_columns) result_cols[i++]->insertFrom(*col, lpos); @@ -1099,7 +1113,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin() } else { - /// skip rows in right table until we find match for current row in left table nextDistinct(*right_cursor); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index c8ba857781e..15bf13381b8 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -167,7 +167,7 @@ public: void set(const FullMergeJoinCursor & rcursor, size_t rpos); void reset(); - bool hasMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) + bool hasMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const { if (value.empty()) return false; diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index e3423aa0386..a3fda006eb8 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -30,31 +32,6 @@ using namespace DB; namespace { -[[ maybe_unused ]] -String dumpBlockSource(std::shared_ptr source, bool mono_block = false) -{ - WriteBufferFromOwnString buf; - { - Block header = source->getPort().getHeader(); - QueryPipeline pipeline(source); - auto format = std::make_shared(buf, header, FormatSettings{}, mono_block); - pipeline.complete(std::move(format)); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - } - return buf.str(); -} - -[[ maybe_unused ]] -String dumpBlock(const Block & block) -{ - Block header = block.cloneEmpty(); - Chunk data(block.getColumns(), block.rows()); - auto source = std::make_shared(header, std::move(data)); - return dumpBlockSource(std::move(source)); -} - QueryPipeline buildJoinPipeline( std::shared_ptr left_source, std::shared_ptr right_source, diff --git a/tests/queries/0_stateless/00927_asof_join_other_types.sql.j2 b/tests/queries/0_stateless/00927_asof_join_other_types.sql.j2 old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01116_asof_join_dolbyzerr.reference b/tests/queries/0_stateless/01116_asof_join_dolbyzerr.reference index 1055a67ea5b..0aa1a85f19d 100644 --- a/tests/queries/0_stateless/01116_asof_join_dolbyzerr.reference +++ b/tests/queries/0_stateless/01116_asof_join_dolbyzerr.reference @@ -1,3 +1,6 @@ v1 o1 ['s2','s1'] v1 o2 ['s4'] v2 o3 ['s5','s3'] +v1 o1 ['s2','s1'] +v1 o2 ['s4'] +v2 o3 ['s5','s3'] diff --git a/tests/queries/0_stateless/01116_asof_join_dolbyzerr.sql b/tests/queries/0_stateless/01116_asof_join_dolbyzerr.sql index 8a94b6ddd24..652cb35cf2a 100644 --- a/tests/queries/0_stateless/01116_asof_join_dolbyzerr.sql +++ b/tests/queries/0_stateless/01116_asof_join_dolbyzerr.sql @@ -16,3 +16,17 @@ GROUP BY ORDER BY visitorId ASC, orderId ASC; + +SELECT + visitorId, + orderId, + groupUniqArray(sessionId) +FROM sessions +ASOF INNER JOIN orders ON (sessions.visitorId = orders.visitorId) AND (sessions.date <= orders.date) +GROUP BY + visitorId, + orderId +ORDER BY + visitorId ASC, + orderId ASC +SETTINGS join_algorithm = 'full_sorting_merge'; diff --git a/tests/queries/0_stateless/02240_asof_join_biginteger.reference b/tests/queries/0_stateless/02240_asof_join_biginteger.reference index cac55eec430..f7eb4d74375 100644 --- a/tests/queries/0_stateless/02240_asof_join_biginteger.reference +++ b/tests/queries/0_stateless/02240_asof_join_biginteger.reference @@ -2,3 +2,7 @@ 0 340282366920938463463374607431768211457 0 18446744073709551617 0 340282366920938463463374607431768211457 +0 18446744073709551617 +0 340282366920938463463374607431768211457 +0 18446744073709551617 +0 340282366920938463463374607431768211457 diff --git a/tests/queries/0_stateless/02240_asof_join_biginteger.sql b/tests/queries/0_stateless/02240_asof_join_biginteger.sql index 6dc5b00f116..a5c1faae4ea 100644 --- a/tests/queries/0_stateless/02240_asof_join_biginteger.sql +++ b/tests/queries/0_stateless/02240_asof_join_biginteger.sql @@ -3,3 +3,11 @@ select * from (select 0 as k, toInt256('340282366920938463463374607431768211457' select * from (select 0 as k, toUInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toUInt128('18446744073709551616') as v) t2 using(k, v); select * from (select 0 as k, toUInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toUInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); + +SET join_algorithm = 'full_sorting_merge'; + +select * from (select 0 as k, toInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toInt128('18446744073709551616') as v) t2 using(k, v); +select * from (select 0 as k, toInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); + +select * from (select 0 as k, toUInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toUInt128('18446744073709551616') as v) t2 using(k, v); +select * from (select 0 as k, toUInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toUInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); diff --git a/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.reference b/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.reference index d00491fd7e5..6ed281c757a 100644 --- a/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.reference +++ b/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.reference @@ -1 +1,2 @@ 1 +1 diff --git a/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.sql b/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.sql index 13dfb5debe7..6aa70a379c1 100644 --- a/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.sql +++ b/tests/queries/0_stateless/02724_function_in_left_table_clause_asof_join.sql @@ -6,3 +6,15 @@ ASOF LEFT JOIN ( select 1 as session_id, 4 as id ) as visitors ON visitors.session_id <= sessions.id AND arrayFirst(a -> a, arrayMap((a) -> a, sessions.arr)) = visitors.id +; + +select count(*) +from ( + select 1 as id, [1, 2, 3] as arr +) as sessions +ASOF LEFT JOIN ( + select 1 as session_id, 4 as id +) as visitors +ON visitors.session_id <= sessions.id AND arrayFirst(a -> a, arrayMap((a) -> a, sessions.arr)) = visitors.id +SETTINGS join_algorithm = 'full_sorting_merge' +; diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.reference b/tests/queries/0_stateless/03143_asof_join_ddb_long.reference new file mode 100644 index 00000000000..2850a8aba98 --- /dev/null +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.reference @@ -0,0 +1,2 @@ +49999983751397 10000032 +49999983751397 10000032 diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql new file mode 100644 index 00000000000..c421702bb00 --- /dev/null +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -0,0 +1,48 @@ +-- Tags: long + +DROP TABLE IF EXISTS build; +DROP TABLE IF EXISTS skewed_probe; + +CREATE TABLE build ENGINE = MergeTree ORDER BY (key, begin) +AS + SELECT + toDateTime('1990-03-21 13:00:00') + INTERVAL number MINUTE AS begin, + number % 4 AS key, + number AS value + FROM numbers(0, 10000000); + +CREATE TABLE skewed_probe ENGINE = MergeTree ORDER BY (key, begin) +AS + SELECT + toDateTime('1990-04-21 13:00:01') + INTERVAL number MINUTE AS begin, + 0 AS key + FROM numbers(0, 5) + UNION ALL + SELECT + toDateTime('1990-05-21 13:00:01') + INTERVAL number MINUTE AS begin, + 1 AS key + FROM numbers(0, 10) + UNION ALL + SELECT + toDateTime('1990-06-21 13:00:01') + INTERVAL number MINUTE AS begin, + 2 AS key + FROM numbers(0, 20) + UNION ALL + SELECT + toDateTime('1990-03-21 13:00:01') + INTERVAL number MINUTE AS begin, + 3 AS key + FROM numbers(0, 10000000); + + +SELECT SUM(value), COUNT(*) +FROM skewed_probe +ASOF JOIN build +USING (key, begin) +; + +SELECT SUM(value), COUNT(*) +FROM skewed_probe +ASOF JOIN build +USING (key, begin) +SETTINGS join_algorithm = 'full_sorting_merge' +; diff --git a/tests/queries/0_stateless/03144_asof_join_ddb_doubles.reference b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.reference new file mode 100644 index 00000000000..f130f0a3f3b --- /dev/null +++ b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.reference @@ -0,0 +1,58 @@ +1 0 +2 0 +3 1 +4 1 +5 1 +6 2 +7 2 +8 3 +9 3 +0 0 +1 0 +2 0 +3 1 +4 1 +5 1 +6 2 +7 2 +8 3 +9 3 +1 1 0 +1 2 0 +1 3 1 +1 4 1 +1 5 1 +1 6 2 +1 7 2 +1 8 3 +1 9 3 +2 0 10 +2 1 10 +2 2 10 +2 3 10 +2 4 10 +2 5 10 +2 6 10 +2 7 20 +2 8 20 +2 9 20 +1 0 0 +1 1 0 +1 2 0 +1 3 1 +1 4 1 +1 5 1 +1 6 2 +1 7 2 +1 8 3 +1 9 3 +2 0 10 +2 1 10 +2 2 10 +2 3 10 +2 4 10 +2 5 10 +2 6 10 +2 7 20 +2 8 20 +2 9 20 diff --git a/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql new file mode 100644 index 00000000000..ef16ced3082 --- /dev/null +++ b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql @@ -0,0 +1,64 @@ +SET join_algorithm = 'full_sorting_merge'; + +DROP TABLE IF EXISTS events0; + +CREATE TABLE events0 ( + begin Float64, + value Int32 +) ENGINE = MergeTree ORDER BY begin; + +INSERT INTO events0 VALUES (1.0, 0), (3.0, 1), (6.0, 2), (8.0, 3); + +SELECT p.ts, e.value +FROM + (SELECT number :: Float64 AS ts FROM numbers(10)) p +ASOF JOIN events0 e +ON p.ts >= e.begin +ORDER BY p.ts ASC; + +SELECT p.ts, e.value +FROM + (SELECT number :: Float64 AS ts FROM numbers(10)) p +ASOF LEFT JOIN events0 e +ON p.ts >= e.begin +ORDER BY p.ts ASC +-- SETTINGS join_use_nulls = 1 +; + +DROP TABLE IF EXISTS events0; + +DROP TABLE IF EXISTS events; +DROP TABLE IF EXISTS probes; + +CREATE TABLE events ( + key Int32, + begin Float64, + value Int32 +) ENGINE = MergeTree ORDER BY (key, begin); + +INSERT INTO events VALUES (1, 1.0, 0), (1, 3.0, 1), (1, 6.0, 2), (1, 8.0, 3), (2, 0.0, 10), (2, 7.0, 20), (2, 11.0, 30); + +CREATE TABLE probes ( + key Int32, + ts Float64 +) ENGINE = MergeTree ORDER BY (key, ts) AS +SELECT + key.number, + ts.number +FROM + numbers(1, 2) as key, + numbers(10) as ts +SETTINGS join_algorithm = 'hash'; + +SELECT p.key, p.ts, e.value +FROM probes p +ASOF JOIN events e +ON p.key = e.key AND p.ts >= e.begin +ORDER BY p.key, p.ts ASC; + +SELECT p.key, p.ts, e.value +FROM probes p +ASOF LEFT JOIN events e +ON p.key = e.key AND p.ts >= e.begin +ORDER BY p.key, p.ts ASC NULLS FIRST; + diff --git a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference new file mode 100644 index 00000000000..73c4f7dfe25 --- /dev/null +++ b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference @@ -0,0 +1,68 @@ +2023-03-21 12:00:00 1970-01-01 00:00:00 -1 +2023-03-21 13:00:00 1970-01-01 00:00:00 -1 +2023-03-21 14:00:00 2023-03-21 13:00:00 0 +2023-03-21 15:00:00 2023-03-21 14:00:00 1 +2023-03-21 16:00:00 2023-03-21 15:00:00 2 +2023-03-21 17:00:00 2023-03-21 16:00:00 3 +2023-03-21 18:00:00 2023-03-21 16:00:00 3 +2023-03-21 19:00:00 2023-03-21 16:00:00 3 +2023-03-21 20:00:00 2023-03-21 16:00:00 3 +2023-03-21 21:00:00 2023-03-21 16:00:00 3 +2027-10-18 11:03:27 2023-03-21 16:00:00 3 +2023-03-21 12:00:00 1970-01-01 00:00:00 -1 +2023-03-21 13:00:00 1970-01-01 00:00:00 -1 +2023-03-21 14:00:00 2023-03-21 13:00:00 0 +2023-03-21 15:00:00 2023-03-21 14:00:00 1 +2023-03-21 16:00:00 2023-03-21 15:00:00 2 +2023-03-21 17:00:00 2023-03-21 16:00:00 3 +2023-03-21 18:00:00 2023-03-21 16:00:00 3 +2023-03-21 19:00:00 2023-03-21 16:00:00 3 +2023-03-21 20:00:00 2023-03-21 16:00:00 3 +2023-03-21 21:00:00 2023-03-21 16:00:00 3 +2027-10-18 11:03:27 2023-03-21 16:00:00 3 +\N \N \N +2023-03-21 12:00:00 2023-03-21 13:00:00 0 +2023-03-21 13:00:00 2023-03-21 13:00:00 0 +2023-03-21 14:00:00 2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2023-03-21 15:00:00 2 +2023-03-21 16:00:00 2023-03-21 16:00:00 3 +2023-03-21 17:00:00 2027-10-18 11:03:27 9 +2023-03-21 18:00:00 2027-10-18 11:03:27 9 +2023-03-21 19:00:00 2027-10-18 11:03:27 9 +2023-03-21 20:00:00 2027-10-18 11:03:27 9 +2023-03-21 21:00:00 2027-10-18 11:03:27 9 +2027-10-18 11:03:27 2027-10-18 11:03:27 9 +2023-03-21 12:00:00 2023-03-21 13:00:00 0 +2023-03-21 13:00:00 2023-03-21 13:00:00 0 +2023-03-21 14:00:00 2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2023-03-21 15:00:00 2 +2023-03-21 16:00:00 2023-03-21 16:00:00 3 +2023-03-21 17:00:00 2027-10-18 11:03:27 9 +2023-03-21 18:00:00 2027-10-18 11:03:27 9 +2023-03-21 19:00:00 2027-10-18 11:03:27 9 +2023-03-21 20:00:00 2027-10-18 11:03:27 9 +2023-03-21 21:00:00 2027-10-18 11:03:27 9 +2027-10-18 11:03:27 2027-10-18 11:03:27 9 +\N \N \N +2023-03-21 12:00:00 2023-03-21 13:00:00 0 +2023-03-21 13:00:00 2023-03-21 14:00:00 1 +2023-03-21 14:00:00 2023-03-21 15:00:00 2 +2023-03-21 15:00:00 2023-03-21 16:00:00 3 +2023-03-21 16:00:00 2027-10-18 11:03:27 9 +2023-03-21 17:00:00 2027-10-18 11:03:27 9 +2023-03-21 18:00:00 2027-10-18 11:03:27 9 +2023-03-21 19:00:00 2027-10-18 11:03:27 9 +2023-03-21 20:00:00 2027-10-18 11:03:27 9 +2023-03-21 21:00:00 2027-10-18 11:03:27 9 +2023-03-21 12:00:00 2023-03-21 13:00:00 0 +2023-03-21 13:00:00 2023-03-21 14:00:00 1 +2023-03-21 14:00:00 2023-03-21 15:00:00 2 +2023-03-21 15:00:00 2023-03-21 16:00:00 3 +2023-03-21 16:00:00 2027-10-18 11:03:27 9 +2023-03-21 17:00:00 2027-10-18 11:03:27 9 +2023-03-21 18:00:00 2027-10-18 11:03:27 9 +2023-03-21 19:00:00 2027-10-18 11:03:27 9 +2023-03-21 20:00:00 2027-10-18 11:03:27 9 +2023-03-21 21:00:00 2027-10-18 11:03:27 9 +2027-10-18 11:03:27 \N \N +\N \N \N diff --git a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql new file mode 100644 index 00000000000..69de17541c1 --- /dev/null +++ b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS events0; +DROP TABLE IF EXISTS probe0; + +SET join_algorithm = 'full_sorting_merge'; + +CREATE TABLE events0 ( + begin Nullable(DateTime('UTC')), + value Int32 +) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO events0 SELECT toDateTime('2023-03-21 13:00:00', 'UTC') + INTERVAL number HOUR, number FROM numbers(4); +INSERT INTO events0 VALUES (NULL, -10),('0000-01-01 00:00:00', -1), ('9999-12-31 23:59:59', 9); + +CREATE TABLE probe0 ( + begin Nullable(DateTime('UTC')) +) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO probe0 SELECT toDateTime('2023-03-21 12:00:00', 'UTC') + INTERVAl number HOUR FROM numbers(10); +INSERT INTO probe0 VALUES (NULL),('9999-12-31 23:59:59'); + +SET join_use_nulls = 1; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF JOIN events0 e +ON p.begin > e.begin +ORDER BY p.begin ASC; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF LEFT JOIN events0 e +ON p.begin > e.begin +ORDER BY p.begin ASC +; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF JOIN events0 e +ON p.begin <= e.begin +ORDER BY p.begin ASC; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF LEFT JOIN events0 e +ON p.begin <= e.begin +ORDER BY p.begin ASC; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF JOIN events0 e +ON p.begin < e.begin +ORDER BY p.begin ASC +; + +SELECT p.begin, e.begin, e.value +FROM probe0 p +ASOF LEFT JOIN events0 e +ON p.begin < e.begin +ORDER BY p.begin ASC; + + +DROP TABLE IF EXISTS events0; +DROP TABLE IF EXISTS probe0; diff --git a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.reference b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.reference new file mode 100644 index 00000000000..ca481c7fff0 --- /dev/null +++ b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.reference @@ -0,0 +1,2 @@ +26790 1488 +26790 1488 diff --git a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 new file mode 100644 index 00000000000..551bac0cc06 --- /dev/null +++ b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 @@ -0,0 +1,37 @@ +SET allow_experimental_analyzer=1; + +SET session_timezone = 'UTC'; + +{% for join_algorithm in ['default', 'full_sorting_merge'] -%} + +SET join_algorithm = '{{ join_algorithm }}'; + +-- TODO: support enable for full_sorting_merge +-- SET join_use_nulls = 1; + +WITH build AS ( + SELECT + tk.number AS k, + toDateTime('2021-01-01 00:00:00') + INTERVAL i.number SECONDS AS t, + i.number % 37 AS v + FROM numbers(3000000) AS i + CROSS JOIN numbers(2) AS tk + SETTINGS join_algorithm = 'hash', join_use_nulls = 0 +), +probe AS ( + SELECT + tk.number AS k, + toDateTime('2021-01-01 00:00:30') + INTERVAL tt.number HOUR AS t + FROM numbers(2) AS tk + CROSS JOIN numbers(toUInt32((toDateTime('2021-02-01 00:00:30') - toDateTime('2021-01-01 00:00:30')) / 3600)) AS tt + SETTINGS join_algorithm = 'hash', join_use_nulls = 0 +) +SELECT + SUM(v) AS v, + COUNT(*) AS n +FROM probe +ASOF LEFT JOIN build +USING (k, t) +; + +{% endfor -%} From 29494d0bc6cc1b0f82a887a06f0643ea4880e681 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 1 May 2024 15:08:50 +0000 Subject: [PATCH 15/74] add asof_join_ddb tests --- .../03144_asof_join_ddb_doubles.sql | 1 + .../03145_asof_join_ddb_inequalities.sql | 1 + .../03147_asof_join_ddb_missing.reference | 10 + .../03147_asof_join_ddb_missing.sql | 186 ++++++++++++++++++ .../03148_asof_join_ddb_subquery.reference | 4 + .../03148_asof_join_ddb_subquery.sql | 29 +++ .../03149_asof_join_ddb_timestamps.reference | 56 ++++++ .../03149_asof_join_ddb_timestamps.sql | 95 +++++++++ 8 files changed, 382 insertions(+) create mode 100644 tests/queries/0_stateless/03147_asof_join_ddb_missing.reference create mode 100644 tests/queries/0_stateless/03147_asof_join_ddb_missing.sql create mode 100644 tests/queries/0_stateless/03148_asof_join_ddb_subquery.reference create mode 100644 tests/queries/0_stateless/03148_asof_join_ddb_subquery.sql create mode 100644 tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference create mode 100644 tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql diff --git a/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql index ef16ced3082..87aece14628 100644 --- a/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql +++ b/tests/queries/0_stateless/03144_asof_join_ddb_doubles.sql @@ -1,4 +1,5 @@ SET join_algorithm = 'full_sorting_merge'; +SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS events0; diff --git a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql index 69de17541c1..ce4badbd597 100644 --- a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql +++ b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS events0; DROP TABLE IF EXISTS probe0; +SET allow_experimental_analyzer = 1; SET join_algorithm = 'full_sorting_merge'; CREATE TABLE events0 ( diff --git a/tests/queries/0_stateless/03147_asof_join_ddb_missing.reference b/tests/queries/0_stateless/03147_asof_join_ddb_missing.reference new file mode 100644 index 00000000000..11eb84463f4 --- /dev/null +++ b/tests/queries/0_stateless/03147_asof_join_ddb_missing.reference @@ -0,0 +1,10 @@ +108 +108 27 +513 +1218 +3528 +14553 +121275 +1495503 +12462525 +1249625025 diff --git a/tests/queries/0_stateless/03147_asof_join_ddb_missing.sql b/tests/queries/0_stateless/03147_asof_join_ddb_missing.sql new file mode 100644 index 00000000000..95a5f8ab3ff --- /dev/null +++ b/tests/queries/0_stateless/03147_asof_join_ddb_missing.sql @@ -0,0 +1,186 @@ +SET allow_experimental_analyzer=1; + +SET session_timezone = 'UTC'; +SET joined_subquery_requires_alias = 0; +SET allow_experimental_analyzer = 1; +SET join_algorithm = 'full_sorting_merge'; + +-- # 10 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(10), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # Coverage: Missing right side bin +WITH build AS ( + SELECT + k * 2 AS k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(10), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + intDiv(k, 2) AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v), COUNT(*) +FROM probe ASOF JOIN build USING (k, t); + +-- # 20 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(20), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 30 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(30), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 50 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(50), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 100 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(100), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 100 dates, 50 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(100), (SELECT number AS k FROM numbers(50)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 1000 dates, 5 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(1000), (SELECT number AS k FROM numbers(5)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 1000 dates, 50 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(1000), (SELECT number AS k FROM numbers(50)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); + +-- # 10000 dates, 50 keys +WITH build AS ( + SELECT + k, + toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t, + number AS v + FROM numbers(10000), (SELECT number AS k FROM numbers(50)) + SETTINGS join_algorithm = 'default' +), +probe AS ( + SELECT + k * 2 AS k, + t - INTERVAL 30 SECOND AS t + FROM build +) +SELECT SUM(v) +FROM probe ASOF JOIN build USING (k, t); diff --git a/tests/queries/0_stateless/03148_asof_join_ddb_subquery.reference b/tests/queries/0_stateless/03148_asof_join_ddb_subquery.reference new file mode 100644 index 00000000000..387a4a8f249 --- /dev/null +++ b/tests/queries/0_stateless/03148_asof_join_ddb_subquery.reference @@ -0,0 +1,4 @@ +1 1 +3 1 +6 1 +8 1 diff --git a/tests/queries/0_stateless/03148_asof_join_ddb_subquery.sql b/tests/queries/0_stateless/03148_asof_join_ddb_subquery.sql new file mode 100644 index 00000000000..2ddf0f09b1e --- /dev/null +++ b/tests/queries/0_stateless/03148_asof_join_ddb_subquery.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS events; +CREATE TABLE events (begin Float64, value Int32) ENGINE = MergeTree() ORDER BY begin; + +INSERT INTO events VALUES (1, 0), (3, 1), (6, 2), (8, 3); + +SET allow_experimental_analyzer = 1; +SET join_algorithm = 'full_sorting_merge'; +SET joined_subquery_requires_alias = 0; + +SELECT + begin, + value IN ( + SELECT e1.value + FROM ( + SELECT * + FROM events e1 + WHERE e1.value = events.value + ) AS e1 + ASOF JOIN ( + SELECT number :: Float64 AS begin + FROM numbers(10) + WHERE number >= 1 AND number < 10 + ) + USING (begin) + ) +FROM events +ORDER BY begin ASC; + +DROP TABLE IF EXISTS events; diff --git a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference new file mode 100644 index 00000000000..7cfc85d23a5 --- /dev/null +++ b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference @@ -0,0 +1,56 @@ +2023-03-21 13:00:00 0 +2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2 +2023-03-21 16:00:00 3 +2023-03-21 17:00:00 3 +2023-03-21 18:00:00 3 +2023-03-21 19:00:00 3 +2023-03-21 20:00:00 3 +2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 +2023-03-21 13:00:00 0 +2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2 +2023-03-21 16:00:00 3 +2023-03-21 17:00:00 3 +2023-03-21 18:00:00 3 +2023-03-21 19:00:00 3 +2023-03-21 20:00:00 3 +2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 +2023-03-21 12:00:00 \N +2023-03-21 13:00:00 0 +2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2 +2023-03-21 16:00:00 3 +2023-03-21 17:00:00 3 +2023-03-21 18:00:00 3 +2023-03-21 19:00:00 3 +2023-03-21 20:00:00 3 +2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 +\N \N +2023-03-21 12:00:00 0 +2023-03-21 13:00:00 0 +2023-03-21 14:00:00 1 +2023-03-21 15:00:00 2 +2023-03-21 16:00:00 3 +2023-03-21 17:00:00 3 +2023-03-21 18:00:00 3 +2023-03-21 19:00:00 3 +2023-03-21 20:00:00 3 +2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 +\N 0 +2023-03-21 12:00:00 \N +2023-03-21 13:00:00 \N +2023-03-21 14:00:00 \N +2023-03-21 15:00:00 \N +2023-03-21 16:00:00 \N +2023-03-21 17:00:00 \N +2023-03-21 18:00:00 \N +2023-03-21 19:00:00 \N +2023-03-21 20:00:00 \N +2023-03-21 21:00:00 \N +2106-02-07 06:28:15 \N +\N \N diff --git a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql new file mode 100644 index 00000000000..ff4518a3775 --- /dev/null +++ b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql @@ -0,0 +1,95 @@ +DROP TABLE IF EXISTS events0; +DROP TABLE IF EXISTS probe0; + +SET session_timezone = 'UTC'; +SET allow_experimental_analyzer = 1; +SET join_algorithm = 'full_sorting_merge'; +SET join_use_nulls = 1; + +CREATE TABLE events0 +ENGINE = MergeTree() +ORDER BY COALESCE(begin, toDateTime('9999-12-31 23:59:59')) +AS +SELECT + toNullable(toDateTime('2023-03-21 13:00:00') + INTERVAL number HOUR) AS begin, + number AS value +FROM numbers(4); + +INSERT INTO events0 VALUES (NULL, -1), (toDateTime('9999-12-31 23:59:59'), 9); + +CREATE TABLE probe0 +ENGINE = MergeTree() +ORDER BY COALESCE(begin, toDateTime('9999-12-31 23:59:59')) +AS +SELECT + toNullable(toDateTime('2023-03-21 12:00:00') + INTERVAL number HOUR) AS begin +FROM numbers(10); + +INSERT INTO probe0 VALUES (NULL), (toDateTime('9999-12-31 23:59:59')); + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF JOIN events0 e ON p.begin >= e.begin +ORDER BY p.begin ASC; + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF JOIN events0 e USING (begin) +ORDER BY p.begin ASC +SETTINGS join_use_nulls = 0 +; + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF LEFT JOIN events0 e ON p.begin >= e.begin +ORDER BY p.begin ASC; + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF LEFT JOIN events0 e USING (begin) +ORDER BY p.begin ASC +SETTINGS join_use_nulls = 0 +; + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF RIGHT JOIN events0 e ON p.begin >= e.begin +ORDER BY e.begin ASC; -- { serverError NOT_IMPLEMENTED} + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF RIGHT JOIN events0 e USING (begin) +ORDER BY e.begin ASC; -- { serverError NOT_IMPLEMENTED} + + +SELECT + p.begin, + e.value +FROM + probe0 p + ASOF LEFT JOIN ( + SELECT * FROM events0 WHERE log(value + 5) > 10 + ) e ON p.begin >= e.begin +ORDER BY p.begin ASC; + + +DROP TABLE IF EXISTS events0; +DROP TABLE IF EXISTS probe0; From d37f03201aaac06b1a175ac0b177c17d945dddbb Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 May 2024 10:07:43 +0000 Subject: [PATCH 16/74] fix --- src/Processors/Transforms/MergeJoinTransform.cpp | 2 -- tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 26e1ebb0b60..9e6904f0613 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1173,9 +1173,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (auto result = handleAllJoinState()) - { return std::move(*result); - } if (auto result = handleAsofJoinState()) return std::move(*result); diff --git a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 index 551bac0cc06..44c54ae2a39 100644 --- a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 +++ b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: long + SET allow_experimental_analyzer=1; SET session_timezone = 'UTC'; From 558b73aba4f89db4a2b3e5ed0754749caddd5dfb Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 May 2024 14:20:40 +0000 Subject: [PATCH 17/74] t --- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index c421702bb00..a41b667e6c8 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,5 +1,7 @@ -- Tags: long +-- https://s3.amazonaws.com/clickhouse-test-reports/55051/07f288862c56b0a98379a07101062689b0460788/stateless_tests_flaky_check__asan_.html + DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From f710a67fb2e72361801f0ae9a45ccbd07e2c7f30 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 22 May 2024 12:27:04 +0000 Subject: [PATCH 18/74] set timezone in 03143_asof_join_ddb_long --- src/Processors/Transforms/MergeJoinTransform.cpp | 4 ++-- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 9e6904f0613..38b63a856f6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -45,8 +45,8 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, bool isNullAt(const IColumn & column, size_t row) { - if (const auto * nullable = checkAndGetColumn(column)) - return nullable->isNullAt(row); + if (const auto * nullable_column = checkAndGetColumn(&column)) + return nullable_column->isNullAt(row); return false; } diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index a41b667e6c8..17a67511030 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,10 +1,10 @@ -- Tags: long --- https://s3.amazonaws.com/clickhouse-test-reports/55051/07f288862c56b0a98379a07101062689b0460788/stateless_tests_flaky_check__asan_.html - DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; +SET session_timezone = 'UTC'; + CREATE TABLE build ENGINE = MergeTree ORDER BY (key, begin) AS SELECT From 11f4ed75fd18a853e70c94f10596d5b3b09a35dd Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 5 Jun 2024 12:26:05 +0200 Subject: [PATCH 19/74] fix build --- src/Processors/tests/gtest_full_sorting_join.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index a3fda006eb8..7294a1b381a 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -151,7 +151,6 @@ public: size_t rows = current_chunk.front()->size(); chunks.emplace_back(std::move(current_chunk), rows); current_chunk = header.cloneEmptyColumns(); - return; } std::shared_ptr getSource() @@ -403,7 +402,7 @@ try auto expected_right = ColumnString::create(); UInt64 k1 = 1; - String k2 = ""; + String k2; auto get_attr = [&](const String & side, size_t idx) -> String { @@ -601,7 +600,7 @@ try ColumnInt64::Container expected; UInt64 k1 = 1; - String k2 = ""; + String k2; auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { @@ -690,7 +689,7 @@ try ColumnInt64::Container expected; UInt64 k1 = 1; - String k2 = ""; + String k2; UInt64 left_t = 0; auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); From 99456d2fa630455ebf1a2422b5e827ec056f4130 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Jun 2024 17:21:15 +0200 Subject: [PATCH 20/74] Fix unexpeced size of low cardinality column in functions --- src/Functions/IFunction.cpp | 101 ++++++++++++++++++++++++------------ 1 file changed, 68 insertions(+), 33 deletions(-) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 31695fc95d5..9217071ca11 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -50,43 +50,78 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args) ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { - size_t num_rows = input_rows_count; + /// We return the LC indexes so the LC can be reconstructed with the function result ColumnPtr indexes; - /// Find first LowCardinality column and replace it to nested dictionary. - for (auto & column : args) + size_t number_low_cardinality_columns = 0; + size_t last_low_cardinality = 0; + size_t number_const_columns = 0; + size_t number_full_columns = 0; + + for (size_t i = 0; i < args.size(); i++) { - if (const auto * low_cardinality_column = checkAndGetColumn(column.column.get())) + auto const & arg = args[i]; + if (checkAndGetColumn(arg.column.get())) { - /// Single LowCardinality column is supported now. - if (indexes) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single dictionary argument for function."); - - const auto * low_cardinality_type = checkAndGetDataType(column.type.get()); - - if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Incompatible type for LowCardinality column: {}", - column.type->getName()); - - if (can_be_executed_on_default_arguments) - { - /// Normal case, when function can be executed on values' default. - column.column = low_cardinality_column->getDictionary().getNestedColumn(); - indexes = low_cardinality_column->getIndexesPtr(); - } - else - { - /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). - /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. - auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); - column.column = dict_encoded.dictionary; - indexes = dict_encoded.indexes; - } - - num_rows = column.column->size(); - column.type = low_cardinality_type->getDictionaryType(); + number_low_cardinality_columns++; + last_low_cardinality = i; } + else if (checkAndGetColumn(arg.column.get())) + number_const_columns++; + else + number_full_columns++; + } + + if (!number_low_cardinality_columns && !number_const_columns) + return nullptr; + + if (number_full_columns > 0 || number_low_cardinality_columns > 1) + { + /// If there is a single full column, we can't replace the LC column with its dictionary, as it won't match + /// the size or order of the full columns. Same if there are 2 or more low cardinality columns + for (auto & arg : args) + { + if (const auto * column_lc = checkAndGetColumn(arg.column.get())) + { + arg.column = recursiveRemoveLowCardinality(arg.column); + chassert(arg.column->size() == input_rows_count); + + const auto * low_cardinality_type = checkAndGetDataType(arg.type.get()); + if (!low_cardinality_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", arg.type->getName()); + arg.type = recursiveRemoveLowCardinality(arg.type); + } + } + } + else if (number_low_cardinality_columns == 1) + { + auto & lc_arg = args[last_low_cardinality]; + + const auto * low_cardinality_type = checkAndGetDataType(lc_arg.type.get()); + if (!low_cardinality_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", lc_arg.type->getName()); + + const auto * low_cardinality_column = checkAndGetColumn(lc_arg.column.get()); + chassert(low_cardinality_column); + + if (can_be_executed_on_default_arguments) + { + /// Normal case, when function can be executed on values' default. + lc_arg.column = low_cardinality_column->getDictionary().getNestedColumn(); + indexes = low_cardinality_column->getIndexesPtr(); + } + else + { + /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). + /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. + auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); + lc_arg.column = dict_encoded.dictionary; + indexes = dict_encoded.indexes; + } + + /// The new column will have a different number of rows, normally less but occasionally it might be more (NULL) + input_rows_count = lc_arg.column->size(); + lc_arg.type = low_cardinality_type->getDictionaryType(); } /// Change size of constants. @@ -94,7 +129,7 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( { if (const auto * column_const = checkAndGetColumn(column.column.get())) { - column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), num_rows); + column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), input_rows_count); column.type = recursiveRemoveLowCardinality(column.type); } } From 1c9df94a0a0b6fc9eb7ef302b47ac909c029e345 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jun 2024 21:26:39 +0200 Subject: [PATCH 21/74] Replace fix with LOGICAL_ERROR --- src/Functions/IFunction.cpp | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 9217071ca11..76ae8f33fbd 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -47,6 +47,9 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args) return true; } +/// Replaces single low cardinality column in a function call by its dictionary +/// This can only happen after the arguments have been adapted in IFunctionOverloadResolver::getReturnType +/// as it's only possible if there is one low cardinality column and, optionally, const columns ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { @@ -77,21 +80,8 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( if (number_full_columns > 0 || number_low_cardinality_columns > 1) { - /// If there is a single full column, we can't replace the LC column with its dictionary, as it won't match - /// the size or order of the full columns. Same if there are 2 or more low cardinality columns - for (auto & arg : args) - { - if (const auto * column_lc = checkAndGetColumn(arg.column.get())) - { - arg.column = recursiveRemoveLowCardinality(arg.column); - chassert(arg.column->size() == input_rows_count); - - const auto * low_cardinality_type = checkAndGetDataType(arg.type.get()); - if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", arg.type->getName()); - arg.type = recursiveRemoveLowCardinality(arg.type); - } - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}", + number_low_cardinality_columns, number_full_columns, number_const_columns); } else if (number_low_cardinality_columns == 1) { @@ -124,7 +114,7 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( lc_arg.type = low_cardinality_type->getDictionaryType(); } - /// Change size of constants. + /// Change size of constants for (auto & column : args) { if (const auto * column_const = checkAndGetColumn(column.column.get())) @@ -305,6 +295,8 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithType bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments(); const auto & dictionary_type = res_low_cardinality_type->getDictionaryType(); + /// The arguments should have been adapted in IFunctionOverloadResolver::getReturnType + /// So there is only one low cardinality column (and optionally some const columns) and no full column ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count); From c185d60375dd0fbc18d062197875ba463326f44a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jun 2024 21:59:14 +0200 Subject: [PATCH 22/74] Try an ugly fix --- src/Planner/PlannerActionsVisitor.cpp | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7a12d5d690d..13e96dc7016 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,16 +485,28 @@ public: return node; } - const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + [[nodiscard]] String addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) { auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end() && it->second->column) + return {node_name}; + if (it != node_name_to_node.end()) - return it->second; + { + /// There is a node with this name, but it doesn't have a column + /// This likely happens because we executed the query until WithMergeableState with a const node in the + /// WHERE clause. As the results of headers are materialized, the column was removed + /// Let's add a new column and keep this + String dupped_name{node_name + "_dupped"}; + const auto * node = &actions_dag.addColumn(column); + node_name_to_node[dupped_name] = node; + return dupped_name; + } const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; - return node; + return {node_name}; } template @@ -723,7 +735,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - actions_stack[0].addConstantIfNecessary(constant_node_name, column); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -732,8 +744,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {constant_node_name, Levels(0)}; - + return {final_name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node) @@ -862,7 +873,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - actions_stack[0].addConstantIfNecessary(column.name, column); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -871,7 +882,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {column.name, Levels(0)}; + return {final_name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) From fed573ffee1bb57f0b9cff7f6f1c7d5a542af51c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 28 Jun 2024 17:12:10 +0200 Subject: [PATCH 23/74] Extend fix --- src/Planner/PlannerActionsVisitor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 13e96dc7016..1c9553032c2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -535,7 +535,7 @@ public: } private: - std::unordered_map node_name_to_node; + std::unordered_map node_name_to_node; ActionsDAG & actions_dag; QueryTreeNodePtr scope_node; }; @@ -741,7 +741,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); } return {final_name, Levels(0)}; @@ -879,7 +879,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); + actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); } return {final_name, Levels(0)}; From 74a5d56f1a404a7271b5a177e4902d0c319195d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 1 Jul 2024 19:32:22 +0200 Subject: [PATCH 24/74] Fix interpolate and add tests --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +- src/Functions/IFunction.cpp | 6 ++ src/Planner/Planner.cpp | 7 +- src/Planner/PlannerActionsVisitor.cpp | 36 ++++++--- src/Planner/PlannerActionsVisitor.h | 9 ++- ..._no_aggregates_and_constant_keys.reference | 4 +- ...nality_group_by_distributed_plan.reference | 55 +++++++++++++ ..._cardinality_group_by_distributed_plan.sql | 80 +++++++++++++++++++ 8 files changed, 185 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference create mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 8860050c5b9..165256479ce 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4100,7 +4100,9 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo auto * column_to_interpolate = interpolate_node_typed.getExpression()->as(); if (!column_to_interpolate) - throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found", + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "INTERPOLATE can work only for identifiers, but {} is found", interpolate_node_typed.getExpression()->formatASTForErrorMessage()); auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName(); diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 76ae8f33fbd..8b092ba9b6e 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -80,8 +80,14 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( if (number_full_columns > 0 || number_low_cardinality_columns > 1) { + /// This should not be possible but currently there are multiple tests in CI failing because of it + /// TODO: Fix those cases, then enable this exception +#if 0 throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}", number_low_cardinality_columns, number_full_columns, number_const_columns); +#else + return nullptr; +#endif } else if (number_low_cardinality_columns == 1) { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2d42ed73223..93a4ea01ff0 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -742,7 +742,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, { auto & interpolate_node_typed = interpolate_node->as(); - PlannerActionsVisitor planner_actions_visitor(planner_context); + PlannerActionsVisitor planner_actions_visitor( + planner_context, + /* use_column_identifier_as_action_node_name_, (default value)*/ true, + /// Prefer the INPUT to CONSTANT nodes (actions must be non constant) + /* prefer_const_column_to_input */ false); + auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 1c9553032c2..a199420c9bd 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,19 +485,24 @@ public: return node; } - [[nodiscard]] String addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + [[nodiscard]] String + addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column, bool prefer_const_column_to_input) { + chassert(column.column != nullptr); auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end() && it->second->column) + if (it != node_name_to_node.end() && (!prefer_const_column_to_input || it->second->column)) return {node_name}; if (it != node_name_to_node.end()) { /// There is a node with this name, but it doesn't have a column /// This likely happens because we executed the query until WithMergeableState with a const node in the - /// WHERE clause. As the results of headers are materialized, the column was removed + /// WHERE clause and, as the results of headers are materialized, the column was removed /// Let's add a new column and keep this String dupped_name{node_name + "_dupped"}; + if (node_name_to_node.find(dupped_name) != node_name_to_node.end()) + return dupped_name; + const auto * node = &actions_dag.addColumn(column); node_name_to_node[dupped_name] = node; return dupped_name; @@ -543,9 +548,11 @@ private: class PlannerActionsVisitorImpl { public: - PlannerActionsVisitorImpl(ActionsDAG & actions_dag, + PlannerActionsVisitorImpl( + ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_); + bool use_column_identifier_as_action_node_name_, + bool prefer_const_column_to_input_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -605,14 +612,18 @@ private: const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; bool use_column_identifier_as_action_node_name; + bool prefer_const_column_to_input; }; -PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag, +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl( + ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_) + bool use_column_identifier_as_action_node_name_, + bool prefer_const_column_to_input_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) + , prefer_const_column_to_input(prefer_const_column_to_input_) { actions_stack.emplace_back(actions_dag, nullptr); } @@ -735,7 +746,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, prefer_const_column_to_input); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -873,7 +884,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - String final_name = actions_stack[0].addConstantIfNecessary(column.name, column); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, prefer_const_column_to_input); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -1019,14 +1030,17 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } -PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_) +PlannerActionsVisitor::PlannerActionsVisitor( + const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, bool prefer_const_column_to_input_) : planner_context(planner_context_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) + , prefer_const_column_to_input(prefer_const_column_to_input_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { - PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); + PlannerActionsVisitorImpl actions_visitor_impl( + actions_dag, planner_context, use_column_identifier_as_action_node_name, prefer_const_column_to_input); return actions_visitor_impl.visit(expression_node); } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 6bb32047327..4bec2d2bb8a 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -27,11 +27,17 @@ using PlannerContextPtr = std::shared_ptr; * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. * 2. For function `in` and its variants, already collected sets from planner context are used. + * 3. When building actions that use CONSTANT nodes, by default we ignore pre-existing INPUTs if those don't have + * a column (a const column always has a column). This is for compatibility with previous headers. We disable this + * behaviour when we explicitly want to override CONSTANT nodes with the input (resolving InterpolateNode for example) */ class PlannerActionsVisitor { public: - explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true); + explicit PlannerActionsVisitor( + const PlannerContextPtr & planner_context_, + bool use_column_identifier_as_action_node_name_ = true, + bool prefer_const_column_to_input_ = true); /** Add actions necessary to calculate expression node into expression dag. * Necessary actions are not added in actions dag output. @@ -42,6 +48,7 @@ public: private: const PlannerContextPtr planner_context; bool use_column_identifier_as_action_node_name = true; + bool prefer_const_column_to_input = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index 63b8a9d14fc..fc77ed8a241 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -8,13 +8,13 @@ 40 41 -0 +41 2 42 2 42 43 -0 +43 11 11 diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference new file mode 100644 index 00000000000..1508c24f410 --- /dev/null +++ b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference @@ -0,0 +1,55 @@ +-- { echoOn } +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM system.one +GROUP BY '666'; +6.666.8 +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM remote('127.0.0.{1,1}', 'system.one') +GROUP BY '666'; +6.666.8 +-- https://github.com/ClickHouse/ClickHouse/issues/63006 +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM system.one +GROUP BY toNullable(6) + WITH ROLLUP +WITH TOTALS; +6 World666666 \N +6 World666666 \N + +6 World666666 \N +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM remote('127.0.0.1') +GROUP BY toNullable(6) + WITH ROLLUP + WITH TOTALS; +6 World666666 \N +6 World666666 \N + +6 World666666 \N +-- { echoOn } +SELECT + '%', + tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), + (toDecimal128(99.67, 6), 36, 61, 14) +FROM dist_03174 +WHERE dummy IN (0, '255') +GROUP BY + toNullable(13), + (99.67, 61, toLowCardinality(14)); +% ('%11default10113%AS%id_02%10101010') (99.67,36,61,14) +-- { echoOn } +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) +FORMAT Null +SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql new file mode 100644 index 00000000000..d397d30e285 --- /dev/null +++ b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql @@ -0,0 +1,80 @@ +-- There are various tests that check that group by keys don't propagate into functions replacing const arguments +-- by full (empty) columns + +DROP TABLE IF EXISTS dist_03174; +DROP TABLE IF EXISTS set_index_not__fuzz_0; + +-- https://github.com/ClickHouse/ClickHouse/issues/63006 + +SET allow_experimental_analyzer=1; + +-- { echoOn } +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM system.one +GROUP BY '666'; + +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM remote('127.0.0.{1,1}', 'system.one') +GROUP BY '666'; + +-- https://github.com/ClickHouse/ClickHouse/issues/63006 +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM system.one +GROUP BY toNullable(6) + WITH ROLLUP +WITH TOTALS; + +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM remote('127.0.0.1') +GROUP BY toNullable(6) + WITH ROLLUP + WITH TOTALS; + +-- https://github.com/ClickHouse/ClickHouse/issues/64945 +-- { echoOff } +CREATE TABLE dist_03174 AS system.one ENGINE = Distributed(test_cluster_two_shards, system, one, dummy); + +-- { echoOn } +SELECT + '%', + tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), + (toDecimal128(99.67, 6), 36, 61, 14) +FROM dist_03174 +WHERE dummy IN (0, '255') +GROUP BY + toNullable(13), + (99.67, 61, toLowCardinality(14)); + +-- Parallel replicas +-- { echoOff } +CREATE TABLE set_index_not__fuzz_0 +( + `name` String, + `status` Enum8('alive' = 0, 'rip' = 1), + INDEX idx_status status TYPE set(2) GRANULARITY 1 +) +ENGINE = MergeTree() +ORDER BY name; + +INSERT INTO set_index_not__fuzz_0 SELECT * FROM generateRandom() LIMIT 10; + +-- { echoOn } +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) +FORMAT Null +SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; + +-- { echoOff } +DROP TABLE IF EXISTS dist_03174; +DROP TABLE IF EXISTS set_index_not__fuzz_0; From 0e559ff7b94c55fe35d0db7174e523180f46e998 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 2 Jul 2024 11:50:43 +0200 Subject: [PATCH 25/74] Better name for flag --- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 26 ++++++++++++++------------ src/Planner/PlannerActionsVisitor.h | 4 ++-- 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 93a4ea01ff0..260fbabf26a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -746,7 +746,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, planner_context, /* use_column_identifier_as_action_node_name_, (default value)*/ true, /// Prefer the INPUT to CONSTANT nodes (actions must be non constant) - /* prefer_const_column_to_input */ false); + /* always_use_const_column_for_constant_nodes */ false); auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index a199420c9bd..e9e1f4edcc2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,12 +485,12 @@ public: return node; } - [[nodiscard]] String - addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column, bool prefer_const_column_to_input) + [[nodiscard]] String addConstantIfNecessary( + const std::string & node_name, const ColumnWithTypeAndName & column, bool always_use_const_column_for_constant_nodes) { chassert(column.column != nullptr); auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end() && (!prefer_const_column_to_input || it->second->column)) + if (it != node_name_to_node.end() && (!always_use_const_column_for_constant_nodes || it->second->column)) return {node_name}; if (it != node_name_to_node.end()) @@ -552,7 +552,7 @@ public: ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, - bool prefer_const_column_to_input_); + bool always_use_const_column_for_constant_nodes_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -612,18 +612,18 @@ private: const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; bool use_column_identifier_as_action_node_name; - bool prefer_const_column_to_input; + bool always_use_const_column_for_constant_nodes; }; PlannerActionsVisitorImpl::PlannerActionsVisitorImpl( ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, - bool prefer_const_column_to_input_) + bool always_use_const_column_for_constant_nodes_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , prefer_const_column_to_input(prefer_const_column_to_input_) + , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) { actions_stack.emplace_back(actions_dag, nullptr); } @@ -746,7 +746,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, prefer_const_column_to_input); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, always_use_const_column_for_constant_nodes); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -884,7 +884,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, prefer_const_column_to_input); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, always_use_const_column_for_constant_nodes); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -1031,16 +1031,18 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } PlannerActionsVisitor::PlannerActionsVisitor( - const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, bool prefer_const_column_to_input_) + const PlannerContextPtr & planner_context_, + bool use_column_identifier_as_action_node_name_, + bool always_use_const_column_for_constant_nodes_) : planner_context(planner_context_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , prefer_const_column_to_input(prefer_const_column_to_input_) + , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { PlannerActionsVisitorImpl actions_visitor_impl( - actions_dag, planner_context, use_column_identifier_as_action_node_name, prefer_const_column_to_input); + actions_dag, planner_context, use_column_identifier_as_action_node_name, always_use_const_column_for_constant_nodes); return actions_visitor_impl.visit(expression_node); } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 4bec2d2bb8a..1dbd149bc4b 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -37,7 +37,7 @@ public: explicit PlannerActionsVisitor( const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true, - bool prefer_const_column_to_input_ = true); + bool always_use_const_column_for_constant_nodes_ = true); /** Add actions necessary to calculate expression node into expression dag. * Necessary actions are not added in actions dag output. @@ -48,7 +48,7 @@ public: private: const PlannerContextPtr planner_context; bool use_column_identifier_as_action_node_name = true; - bool prefer_const_column_to_input = true; + bool always_use_const_column_for_constant_nodes = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. From a84dd6b7710da54e89722318755d4e4e70984e2b Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 10:37:00 +0000 Subject: [PATCH 26/74] Fix some review comments --- .../Transforms/MergeJoinTransform.cpp | 2 +- .../tests/gtest_full_sorting_join.cpp | 48 +++++++++++++------ .../00927_asof_join_correct_bt.reference | 8 ++++ .../00927_asof_join_correct_bt.sql | 3 ++ .../02276_full_sort_join_unsupported.sql | 2 - ...03145_asof_join_ddb_inequalities.reference | 5 ++ .../03145_asof_join_ddb_inequalities.sql | 12 +++-- .../03146_asof_join_ddb_merge_long.sql.j2 | 2 +- .../03149_asof_join_ddb_timestamps.sql | 2 +- 9 files changed, 59 insertions(+), 25 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 38a59cd6d9a..3b69ddaec06 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -43,7 +43,7 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, return std::make_unique(block, desc, strictness == JoinStrictness::Asof); } -bool isNullAt(const IColumn & column, size_t row) +bool ALWAYS_INLINE isNullAt(const IColumn & column, size_t row) { if (const auto * nullable_column = checkAndGetColumn(&column)) return nullable_column->isNullAt(row); diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index 7294a1b381a..f678d7984e8 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -484,7 +484,7 @@ try right_source.addRow({"AAPL", 2, 98}); right_source.addRow({"AAPL", 3, 99}); right_source.addRow({"AMZN", 1, 100}); - right_source.addRow({"AMZN", 2, 0}); + right_source.addRow({"AMZN", 2, 110}); right_source.addChunk(); right_source.addRow({"AMZN", 2, 110}); right_source.addChunk(); @@ -574,12 +574,15 @@ catch (Exception & e) TEST_F(FullSortingJoinTest, AsofLessGeneratedTestData) try { - auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); + /// Generate data random and build expected result at the same time. + /// Test specific combinations of join kind and inequality per each run + auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals }); SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); + /// Key is complex, `k1, k2` for equality and `t` for asof SourceChunksBuilder left_source_builder({ {std::make_shared(), "k1"}, {std::make_shared(), "k2"}, @@ -594,9 +597,11 @@ try {std::make_shared(), "attr"}, }); + /// How small generated block should be left_source_builder.setBreakProbability(rng); right_source_builder.setBreakProbability(rng); + /// We are going to generate sorted data and remember expected result ColumnInt64::Container expected; UInt64 k1 = 1; @@ -604,29 +609,34 @@ try auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { + /// Generate new key greater than previous generateNextKey(rng, k1, k2); Int64 left_t = 0; + /// Generate several rows for the key size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng); for (size_t i = 0; i < num_left_rows; ++i) { + /// t is strictly greater than previous left_t += std::uniform_int_distribution<>(1, 10)(rng); - left_source_builder.addRow({k1, k2, left_t, 10 * left_t}); - expected.push_back(10 * left_t); + auto left_arrtibute_value = 10 * left_t; + left_source_builder.addRow({k1, k2, left_t, left_arrtibute_value}); + expected.push_back(left_arrtibute_value); auto num_matches = 1 + std::poisson_distribution<>(4)(rng); - + /// Generate several matches in the right table auto right_t = left_t; for (size_t j = 0; j < num_matches; ++j) { int min_step = isStrict(asof_inequality) ? 1 : 0; right_t += std::uniform_int_distribution<>(min_step, 3)(rng); + /// First row should match bool is_match = j == 0; - right_source_builder.addRow({k1, k2, right_t, is_match ? 100 * left_t : -1}); + right_source_builder.addRow({k1, k2, right_t, is_match ? 10 * left_arrtibute_value : -1}); } - /// next left_t should be greater than right_t not to match with previous rows + /// Next left_t should be greater than right_t not to match with previous rows left_t = right_t; } @@ -650,7 +660,9 @@ try assertColumnVectorEq(expected, result_block, "t1.attr"); for (auto & e : expected) - e = e < 0 ? 0 : 10 * e; /// non matched rows from left table have negative attr + /// Non matched rows from left table have negative attr + /// Value if attribute in right table is 10 times greater than in left table + e = e < 0 ? 0 : 10 * e; assertColumnVectorEq(expected, result_block, "t2.attr"); } @@ -663,8 +675,10 @@ catch (Exception & e) TEST_F(FullSortingJoinTest, AsofGreaterGeneratedTestData) try { - auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); + /// Generate data random and build expected result at the same time. + /// Test specific combinations of join kind and inequality per each run + auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left }); auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals }); SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality)); @@ -695,9 +709,10 @@ try auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng); for (size_t key_num = 0; key_num < key_num_total; ++key_num) { + /// Generate new key greater than previous generateNextKey(rng, k1, k2); - /// generate some rows with smaller left_t to check that they are not matched + /// Generate some rows with smaller left_t to check that they are not matched size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 100)(rng) : 0; for (size_t i = 0; i < num_left_rows; ++i) { @@ -713,21 +728,22 @@ try size_t num_right_matches = std::uniform_int_distribution<>(1, 10)(rng); auto right_t = left_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 0 : 1, 10)(rng); + auto attribute_value = 10 * right_t; for (size_t j = 0; j < num_right_matches; ++j) { right_t += std::uniform_int_distribution<>(0, 3)(rng); bool is_match = j == num_right_matches - 1; - right_source_builder.addRow({k1, k2, right_t, is_match ? 100 * right_t : -1}); + right_source_builder.addRow({k1, k2, right_t, is_match ? 10 * attribute_value : -1}); } - /// next left_t should be greater than (or equals) right_t to match with previous rows + /// Next left_t should be greater than (or equals) right_t to match with previous rows left_t = right_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 1 : 0, 100)(rng); size_t num_left_matches = std::uniform_int_distribution<>(1, 100)(rng); for (size_t j = 0; j < num_left_matches; ++j) { left_t += std::uniform_int_distribution<>(0, 3)(rng); - left_source_builder.addRow({k1, k2, left_t, 10 * right_t}); - expected.push_back(10 * right_t); + left_source_builder.addRow({k1, k2, left_t, attribute_value}); + expected.push_back(attribute_value); } } @@ -739,7 +755,9 @@ try assertColumnVectorEq(expected, result_block, "t1.attr"); for (auto & e : expected) - e = e < 0 ? 0 : 10 * e; /// non matched rows from left table have negative attr + /// Non matched rows from left table have negative attr + /// Value if attribute in right table is 10 times greater than in left table + e = e < 0 ? 0 : 10 * e; assertColumnVectorEq(expected, result_block, "t2.attr"); } diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.reference b/tests/queries/0_stateless/00927_asof_join_correct_bt.reference index a398f9604fd..28c48d2e290 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.reference +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.reference @@ -1,28 +1,36 @@ +-- { echoOn } +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +SET join_algorithm = 'full_sorting_merge'; +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 1 104 4 4 104 1 1 105 5 4 104 1 +SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); 1 101 1 0 0 0 1 102 2 2 102 1 1 103 3 2 102 1 diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql index 761d6bacde6..d796b62d3b3 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql @@ -13,6 +13,7 @@ INSERT INTO B2(k,t,b) VALUES (1,102,2), (1,104,4); CREATE TABLE B3(k UInt32, b UInt64, t UInt32) ENGINE = MergeTree() ORDER BY (k, t); INSERT INTO B3(k,t,b) VALUES (1,102,2), (1,104,4); +-- { echoOn } SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); @@ -22,6 +23,8 @@ SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t); +-- { echoOff } + DROP TABLE B1; DROP TABLE B2; DROP TABLE B3; diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql index 03936107563..0b10101d8f2 100644 --- a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -19,8 +19,6 @@ SELECT * FROM t1 ANTI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENT SELECT * FROM t1 SEMI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } --- SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serverError NOT_IMPLEMENTED } - SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 JOIN t2 USING (key) SETTINGS join_use_nulls = 1; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference index 73c4f7dfe25..4aac918c98c 100644 --- a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference +++ b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.reference @@ -1,3 +1,4 @@ +- 2023-03-21 12:00:00 1970-01-01 00:00:00 -1 2023-03-21 13:00:00 1970-01-01 00:00:00 -1 2023-03-21 14:00:00 2023-03-21 13:00:00 0 @@ -9,6 +10,7 @@ 2023-03-21 20:00:00 2023-03-21 16:00:00 3 2023-03-21 21:00:00 2023-03-21 16:00:00 3 2027-10-18 11:03:27 2023-03-21 16:00:00 3 +- 2023-03-21 12:00:00 1970-01-01 00:00:00 -1 2023-03-21 13:00:00 1970-01-01 00:00:00 -1 2023-03-21 14:00:00 2023-03-21 13:00:00 0 @@ -32,6 +34,7 @@ 2023-03-21 20:00:00 2027-10-18 11:03:27 9 2023-03-21 21:00:00 2027-10-18 11:03:27 9 2027-10-18 11:03:27 2027-10-18 11:03:27 9 +- 2023-03-21 12:00:00 2023-03-21 13:00:00 0 2023-03-21 13:00:00 2023-03-21 13:00:00 0 2023-03-21 14:00:00 2023-03-21 14:00:00 1 @@ -44,6 +47,7 @@ 2023-03-21 21:00:00 2027-10-18 11:03:27 9 2027-10-18 11:03:27 2027-10-18 11:03:27 9 \N \N \N +- 2023-03-21 12:00:00 2023-03-21 13:00:00 0 2023-03-21 13:00:00 2023-03-21 14:00:00 1 2023-03-21 14:00:00 2023-03-21 15:00:00 2 @@ -54,6 +58,7 @@ 2023-03-21 19:00:00 2027-10-18 11:03:27 9 2023-03-21 20:00:00 2027-10-18 11:03:27 9 2023-03-21 21:00:00 2027-10-18 11:03:27 9 +- 2023-03-21 12:00:00 2023-03-21 13:00:00 0 2023-03-21 13:00:00 2023-03-21 14:00:00 1 2023-03-21 14:00:00 2023-03-21 15:00:00 2 diff --git a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql index ce4badbd597..d67aa254bd6 100644 --- a/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql +++ b/tests/queries/0_stateless/03145_asof_join_ddb_inequalities.sql @@ -21,18 +21,19 @@ INSERT INTO probe0 VALUES (NULL),('9999-12-31 23:59:59'); SET join_use_nulls = 1; +SELECT '-'; SELECT p.begin, e.begin, e.value FROM probe0 p ASOF JOIN events0 e ON p.begin > e.begin ORDER BY p.begin ASC; +SELECT '-'; SELECT p.begin, e.begin, e.value FROM probe0 p ASOF LEFT JOIN events0 e ON p.begin > e.begin -ORDER BY p.begin ASC -; +ORDER BY p.begin ASC; SELECT p.begin, e.begin, e.value FROM probe0 p @@ -40,25 +41,26 @@ ASOF JOIN events0 e ON p.begin <= e.begin ORDER BY p.begin ASC; +SELECT '-'; SELECT p.begin, e.begin, e.value FROM probe0 p ASOF LEFT JOIN events0 e ON p.begin <= e.begin ORDER BY p.begin ASC; +SELECT '-'; SELECT p.begin, e.begin, e.value FROM probe0 p ASOF JOIN events0 e ON p.begin < e.begin -ORDER BY p.begin ASC -; +ORDER BY p.begin ASC; +SELECT '-'; SELECT p.begin, e.begin, e.value FROM probe0 p ASOF LEFT JOIN events0 e ON p.begin < e.begin ORDER BY p.begin ASC; - DROP TABLE IF EXISTS events0; DROP TABLE IF EXISTS probe0; diff --git a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 index 44c54ae2a39..49ba70c471e 100644 --- a/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 +++ b/tests/queries/0_stateless/03146_asof_join_ddb_merge_long.sql.j2 @@ -8,7 +8,7 @@ SET session_timezone = 'UTC'; SET join_algorithm = '{{ join_algorithm }}'; --- TODO: support enable for full_sorting_merge +-- TODO: enable once USING and `join_use_nulls` is supported by `full_sorting_merge` -- SET join_use_nulls = 1; WITH build AS ( diff --git a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql index ff4518a3775..cd83d62dc70 100644 --- a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql +++ b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.sql @@ -87,7 +87,7 @@ FROM probe0 p ASOF LEFT JOIN ( SELECT * FROM events0 WHERE log(value + 5) > 10 - ) e ON p.begin >= e.begin + ) e ON p.begin + INTERVAL 2 HOUR >= e.begin + INTERVAL 1 HOUR ORDER BY p.begin ASC; From a61304508f07694354c1f698ad8c8c6ba5b65edc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jul 2024 21:47:09 +0200 Subject: [PATCH 27/74] Remove noise from unit tests --- src/Common/Exception.h | 9 +++++++++ src/Common/tests/gtest_shell_command.cpp | 13 ------------- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 4 ++++ src/Storages/MergeTree/tests/gtest_executor.cpp | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index a4774a89f6a..4e54c411bf1 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -244,6 +244,15 @@ private: const char * className() const noexcept override { return "DB::ErrnoException"; } }; +/// An exception to use in unit tests to test interfaces. +/// It is distinguished from others, so it does not have to be logged. +class TestException : public Exception +{ +public: + using Exception::Exception; +}; + + using Exceptions = std::vector; /** Try to write an exception to the log (and forget about it). diff --git a/src/Common/tests/gtest_shell_command.cpp b/src/Common/tests/gtest_shell_command.cpp index d6d0a544e9b..0ea96da9da2 100644 --- a/src/Common/tests/gtest_shell_command.cpp +++ b/src/Common/tests/gtest_shell_command.cpp @@ -54,16 +54,3 @@ TEST(ShellCommand, ExecuteWithInput) EXPECT_EQ(res, "Hello, world!\n"); } - -TEST(ShellCommand, AutoWait) -{ - // hunting: - for (int i = 0; i < 1000; ++i) - { - auto command = ShellCommand::execute("echo " + std::to_string(i)); - //command->wait(); // now automatic - } - - // std::cerr << "inspect me: ps auxwwf\n"; - // std::this_thread::sleep_for(std::chrono::seconds(100)); -} diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 8cb0badc19b..4e62c503d65 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -155,6 +155,10 @@ void printExceptionWithRespectToAbort(LoggerPtr log, const String & query_id) { std::rethrow_exception(ex); } + catch (const TestException &) + { + /// Exception from a unit test, ignore it. + } catch (const Exception & e) { NOEXCEPT_SCOPE({ diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index 6f34eb4dfbd..c7057ce87c6 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -34,7 +34,7 @@ public: auto choice = distribution(generator); if (choice == 0) - throw std::runtime_error("Unlucky..."); + throw TestException(); return false; } @@ -48,7 +48,7 @@ public: { auto choice = distribution(generator); if (choice == 0) - throw std::runtime_error("Unlucky..."); + throw TestException(); } Priority getPriority() const override { return {}; } From 2bc65fe2080260d16e27df965610197a38052705 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:44:05 +0000 Subject: [PATCH 28/74] Make ColumnSparse::updateWeakHash32 consistent with internal column. --- src/Columns/ColumnSparse.cpp | 38 +++++++++++++++++++++++++++++++++--- src/Columns/ColumnSparse.h | 3 +++ 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 809586d8810..ea4d23c1678 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -684,13 +685,26 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " "column size is {}, hash size is {}", _size, hash.getData().size()); - auto offset_it = begin(); + size_t values_size = values->size(); + WeakHash32 values_hash(values_size); + auto & hash_data = hash.getData(); + auto & values_hash_data = values_hash.getData(); + const auto & offsets_data = getOffsetsData(); + + if (getNumberOfDefaultRows() > 0) + values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; + + for (size_t i = 0; i < values_size; ++i) + values_hash_data[i + 1] = hash_data[offsets_data[i]]; + + values->updateWeakHash32(values_hash); + + auto offset_it = begin(); for (size_t i = 0; i < _size; ++i, ++offset_it) { size_t value_index = offset_it.getValueIndex(); - auto data_ref = values->getDataAt(value_index); - hash_data[i] = ::updateWeakHash32(reinterpret_cast(data_ref.data), data_ref.size, hash_data[i]); + hash_data[i] = values_hash_data[value_index]; } } @@ -807,6 +821,24 @@ size_t ColumnSparse::getValueIndex(size_t n) const return it - offsets_data.begin() + 1; } +size_t ColumnSparse::getFirstDefaultValueIndex() const +{ + if (getNumberOfDefaultRows() == 0) + return size(); + + const auto & offsets_data = getOffsetsData(); + size_t off_size = offsets_data.size(); + + if (off_size == 0 || offsets_data[0] > 0) + return 0; + + size_t idx = 0; + while (idx + 1 < off_size && offsets_data[idx] + 1 == offsets_data[idx + 1]) + ++idx; + + return offsets_data[idx] + 1; +} + ColumnSparse::Iterator ColumnSparse::getIterator(size_t n) const { const auto & offsets_data = getOffsetsData(); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 3e34d1de94a..4860f5171f7 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -173,6 +173,9 @@ public: /// O(log(offsets.size())) complexity, size_t getValueIndex(size_t n) const; + /// Returns an index of the first default value, or size() if there is no defaults. + size_t getFirstDefaultValueIndex() const; + const IColumn & getValuesColumn() const { return *values; } IColumn & getValuesColumn() { return *values; } From 24b9e1885216a62294e68a977437b4d4c62a12f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:50:27 +0000 Subject: [PATCH 29/74] Add a test. --- ...l_window_finctions_and_column_sparce_bug.reference | 8 ++++++++ ...arallel_window_finctions_and_column_sparce_bug.sql | 11 +++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference create mode 100644 tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference new file mode 100644 index 00000000000..f11ec57a425 --- /dev/null +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference @@ -0,0 +1,8 @@ +false 1 1 +true 1 1 +--- +false 1 1 +false 1 2 +false 1 3 +true 1 1 +true 1 2 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql new file mode 100644 index 00000000000..a4c0200813c --- /dev/null +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql @@ -0,0 +1,11 @@ +create table t(c Int32, d Bool) Engine=MergeTree order by c; +system stop merges t; + +insert into t values (1, 0); +insert into t values (1, 0); +insert into t values (1, 1); +insert into t values (1, 0)(1, 1); + +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; +SELECT '---'; +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; From 902e548a2daf087bdb4363694a91bf665a7f7f9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:52:00 +0000 Subject: [PATCH 30/74] Fix typo. --- ...205_parallel_window_finctions_and_column_sparse_bug.reference} | 0 ... => 03205_parallel_window_finctions_and_column_sparse_bug.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03205_parallel_window_finctions_and_column_sparce_bug.reference => 03205_parallel_window_finctions_and_column_sparse_bug.reference} (100%) rename tests/queries/0_stateless/{03205_parallel_window_finctions_and_column_sparce_bug.sql => 03205_parallel_window_finctions_and_column_sparse_bug.sql} (100%) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference similarity index 100% rename from tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference rename to tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql similarity index 100% rename from tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql rename to tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql From 04f3c29b60658938a93aa6de6f15f8c50121e53e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:47:38 +0000 Subject: [PATCH 31/74] Fix crash. --- src/Columns/ColumnSparse.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index ea4d23c1678..0922eb5ea2d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -695,7 +695,7 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const if (getNumberOfDefaultRows() > 0) values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; - for (size_t i = 0; i < values_size; ++i) + for (size_t i = 0; i + 1 < values_size; ++i) values_hash_data[i + 1] = hash_data[offsets_data[i]]; values->updateWeakHash32(values_hash); From 8acc5d90ca295c2ff6b574da15dec70268a19015 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:47:58 +0000 Subject: [PATCH 32/74] Add more test cases. --- ..._finctions_and_column_sparse_bug.reference | 75 ++++++++++++++++++ ...window_finctions_and_column_sparse_bug.sql | 76 +++++++++++++++++++ 2 files changed, 151 insertions(+) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference index f11ec57a425..356329a392d 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference @@ -6,3 +6,78 @@ false 1 2 false 1 3 true 1 1 true 1 2 +--- +-755809149 0 +--- +1 -2081147898 +1 -1981899149 +2 -2051538534 +2 -1650266905 +3 -1975508531 +3 -1646738223 +4 -1700730666 +4 -1618912877 +5 -1465484835 +5 -1317193174 +6 -1458338029 +6 -1219769753 +7 -1450619195 +7 -1154269118 +8 -1365934326 +8 -1150980622 +9 -1203382363 +9 -1098155311 +10 -1197430632 +10 -841067875 +11 -1176267855 +11 -816935497 +12 -1020892864 +12 -599948807 +13 -991301833 +13 -526570556 +14 -685902265 +14 -504713125 +15 -653505826 +15 -411038390 +16 -451392958 +16 -331834394 +17 -262516786 +17 -176934810 +18 -222873194 +18 -2 +19 -153185515 +19 6 +20 -74234560 +20 255 +21 -41 +21 406615258 +22 -6 +22 541758331 +23 -5 +23 720985423 +24 -3 +24 745669725 +25 15 +25 897064234 +26 65535 +26 1116921321 +27 77089559 +27 1207796283 +28 100663045 +28 1603772265 +29 561061873 +29 1664059402 +30 643897141 +30 1688303275 +31 914629990 +31 1913361922 +32 1159852204 +32 1929066636 +33 1258218855 +33 1968095908 +34 1459407556 +34 2054878592 +35 1936334332 +35 2125075305 +36 1962246186 +37 2030467062 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index a4c0200813c..6e326d0a67f 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -9,3 +9,79 @@ insert into t values (1, 0)(1, 1); SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; SELECT '---'; SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; +SELECT '---'; + +drop table t; + +create table t ( + c Int32 primary key , + s Bool , + w Float64 + ); + +system stop merges t; + +insert into t values(439499072,true,0),(1393290072,true,0); +insert into t values(-1317193174,false,0),(1929066636,false,0); +insert into t values(-2,false,0),(1962246186,true,0),(2054878592,false,0); +insert into t values(-1893563136,true,41.55); +insert into t values(-1338380855,true,-0.7),(-991301833,true,0),(-755809149,false,43.18),(-41,true,0),(3,false,0),(255,false,0),(255,false,0),(189195893,false,0),(195550885,false,9223372036854776000); + +SELECT * FROM ( +SELECT c, min(w) OVER (PARTITION BY s ORDER BY c ASC, s ASC, w ASC) +FROM t limit toUInt64(-1)) +WHERE c = -755809149; + +SELECT '---'; + +create table t_vkx4cc ( + c_ylzjpt Int32, + c_hqfr9 Bool , + ) engine = MergeTree order by c_ylzjpt; + +system stop merges t_vkx4cc; + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2081147898, coalesce((NOT NOT(cast( (53 < 539704722) as Nullable(Bool)))), true)), (-1219769753, coalesce((true) and (false), false)), (-1981899149, coalesce(false, false)), (-1646738223, coalesce((NOT NOT(cast( (23.5 <= -26) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (255, coalesce(false, false)), (-1317193174, coalesce(false, false)), (-41, coalesce(true, false)), (1929066636, coalesce(false, true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-1700730666, coalesce((NOT NOT(cast( (-2022515471055597472 AND -29) as Nullable(Bool)))), false)), (1664059402, coalesce((NOT NOT(cast( (-19643 >= -122) as Nullable(Bool)))), false)), (1688303275, coalesce((NOT NOT(cast( (737275892 < 105) as Nullable(Bool)))), true)), (406615258, coalesce((NOT NOT(cast( (-657730213 = 82.86) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2, coalesce(false, false)), (1962246186, coalesce(true, false)), (-991301833, coalesce(true, true)), (2054878592, coalesce(false, false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (643897141, coalesce((NOT NOT(cast( (-60 AND cast(null as Nullable(Int64))) as Nullable(Bool)))), true)), (-2051538534, coalesce(((-1616816511 between 332225780 and -1883087387)) or ((-573375170 between -1427445977 and 615586748)), false)), (77089559, coalesce((NOT NOT(cast( ((true) and (true) != 925456787) as Nullable(Bool)))), false)), (1116921321, coalesce((0 is NULL), true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-816935497, coalesce(false, false)), (1207796283, coalesce((-129 between -5 and -5), false)), (-1365934326, coalesce(true, false)), (-1618912877, coalesce((NOT NOT(cast( (false >= 31833) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-331834394, coalesce((NOT NOT(cast( (-63 <= -1822810052) as Nullable(Bool)))), true)), (-1020892864, coalesce((NOT NOT(cast( (40.31 <= 8146037365746019777) as Nullable(Bool)))), true)), (-1150980622, coalesce(((94019304 between -730556489 and 32)) and ((-956354236 is not NULL)), true)), (-1203382363, coalesce(true, true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-653505826, coalesce((true) or (true), false)), (-1975508531, coalesce(((-796885845 between 65536 and cast(null as Nullable(Int32)))) or ((NOT NOT(cast( (-7467729336434250795 < 100.20) as Nullable(Bool))))), false)), (-1465484835, coalesce(((NOT NOT(cast( (19209 <= 75.96) as Nullable(Bool))))) or (true), false)), (1968095908, coalesce((NOT NOT(cast( (-1309960412156062327 > 13102) as Nullable(Bool)))), true)); + +alter table t_vkx4cc add column c_zosphq2t1 Float64; + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-153185515, coalesce((NOT NOT(cast( (1291639145 >= 30.22) as Nullable(Bool)))), false), -1.8), (-411038390, coalesce(((-762326135 between 16 and 177530758)) or (false), true), 26.34), (914629990, coalesce((-1125832977 is not NULL), true), 59.2), (541758331, coalesce(false, true), -255.1); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (2125075305, coalesce(false, false), 55.36), (-1176267855, coalesce(true, true), 55.45), (1459407556, coalesce((true) and ((NOT NOT(cast( (95.96 != 65) as Nullable(Bool))))), true), 85.80), (-1098155311, coalesce(false, false), 2147483649.9); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (6, coalesce((NOT NOT(cast( (1546334968 < -4) as Nullable(Bool)))), true), 57.42), (-5, coalesce((NOT NOT(cast( (59 AND 13) as Nullable(Bool)))), false), 65536.3), (100663045, coalesce((-1190355242 is not NULL), true), 73.80), (-451392958, coalesce((NOT NOT(cast( (false != -443845933) as Nullable(Bool)))), false), -4294967294.0); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (561061873, coalesce(true, false), 12.17), (-526570556, coalesce(false, false), 64.73), (-1450619195, coalesce(true, true), 54.33), (-3, coalesce(true, true), 52.9); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-504713125, coalesce(false, true), 27.58), (897064234, coalesce((836516994 between cast(null as Nullable(Int32)) and -1832647080), true), 9223372036854775809.2), (65535, coalesce(true, true), 4294967297.5), (-599948807, coalesce((false) or ((NOT NOT(cast( (6.52 = 65.49) as Nullable(Bool))))), false), 256.5); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1650266905, coalesce((NOT NOT(cast( (-83 = -218055084) as Nullable(Bool)))), true), 1.9), (-841067875, coalesce(false, true), -126.5), (15, coalesce(((NOT NOT(cast( (cast(null as Nullable(Decimal)) = cast(null as Nullable(Int32))) as Nullable(Bool))))) or (true), true), 33.65), (1913361922, coalesce((NOT NOT(cast( (false AND 0) as Nullable(Bool)))), false), 6.4); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1159852204, coalesce((-2057115045 is not NULL), false), 20.61), (-6, coalesce(true, true), 66.33), (-1154269118, coalesce(false, true), 8.89), (1258218855, coalesce(true, false), 19.80); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1603772265, coalesce(false, true), 57.87), (-176934810, coalesce(false, true), 128.8), (-1458338029, coalesce((NOT NOT(cast( (20908 != (NOT NOT(cast( (cast(null as Nullable(Decimal)) <= (true) or ((NOT NOT(cast( (973511022 <= -112) as Nullable(Bool)))))) as Nullable(Bool))))) as Nullable(Bool)))), true), 76.54), (-262516786, coalesce((cast(null as Nullable(Int32)) is NULL), false), 21.49); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1197430632, coalesce(true, false), 45.40), (-685902265, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) < cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 5.55), (1936334332, coalesce((-1565552735 is not NULL), false), 26.28), (2030467062, coalesce((NOT NOT(cast( (127.3 != cast(null as Nullable(Int32))) as Nullable(Bool)))), true), 89.50); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (720985423, coalesce((NOT NOT(cast( (-451448940 = cast(null as Nullable(Decimal))) as Nullable(Bool)))), false), 52.65), (-222873194, coalesce(((-20 between -1419620477 and 1616455043)) or ((25624502 between 1312431316 and 1757361651)), false), 127.2), (745669725, coalesce((NOT NOT(cast( ((NOT NOT(cast( (cast(null as Nullable(UInt64)) <= 42) as Nullable(Bool)))) >= 3233811255032796928) as Nullable(Bool)))), false), 7.74), (-74234560, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) >= cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 19.25); + +SELECT DISTINCT + count(ref_0.c_zosphq2t1) over (partition by ref_0.c_hqfr9 order by ref_0.c_ylzjpt, ref_0.c_hqfr9, ref_0.c_zosphq2t1) as c0, + ref_0.c_ylzjpt as c1 +FROM + t_vkx4cc as ref_0 + order by c0, c1; From 4dea89df763fb9504cb681379d46830f4ec98db3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:50:58 +0000 Subject: [PATCH 33/74] Cleanup. --- src/Columns/ColumnSparse.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0922eb5ea2d..98a66e87387 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include From 67f5ffc5920892b34b1c8b7176337c39cafe7141 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 16 Jul 2024 20:02:53 +0000 Subject: [PATCH 34/74] Fixups --- tests/performance/final_big_column.xml | 4 ++-- tests/performance/function_tokens.xml | 4 ++-- tests/performance/polymorphic_parts_s.xml | 8 ++++---- tests/performance/scripts/report.py | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 1fd586d2d90..5225b3d7ad4 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -10,8 +10,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(2500000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(2500000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null diff --git a/tests/performance/function_tokens.xml b/tests/performance/function_tokens.xml index 1ff56323d62..bc2bc71a933 100644 --- a/tests/performance/function_tokens.xml +++ b/tests/performance/function_tokens.xml @@ -1,5 +1,5 @@ with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByChar(' ', materialize(s)) as w from numbers(1000000) - with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp(' ', materialize(s)) as w from numbers(1000000) - with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp('\s+', materialize(s)) as w from numbers(100000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp(' ', materialize(s)) as w from numbers(200000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp('\s+', materialize(s)) as w from numbers(20000) diff --git a/tests/performance/polymorphic_parts_s.xml b/tests/performance/polymorphic_parts_s.xml index b4dd87a7ae3..5fe1ffffe1d 100644 --- a/tests/performance/polymorphic_parts_s.xml +++ b/tests/performance/polymorphic_parts_s.xml @@ -24,10 +24,10 @@ 1 - - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(1000) - INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(100) + + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(50) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(500) + INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(50) DROP TABLE IF EXISTS hits_wide DROP TABLE IF EXISTS hits_compact diff --git a/tests/performance/scripts/report.py b/tests/performance/scripts/report.py index c2bc773bd54..e45d709ca2c 100755 --- a/tests/performance/scripts/report.py +++ b/tests/performance/scripts/report.py @@ -555,7 +555,7 @@ if args.report == "main": "Total client time for measured query runs, s", # 2 "Queries", # 3 "Longest query, total for measured runs, s", # 4 - "Wall clock time per query, s", # 5 + "Average query wall clock time, s", # 5 "Shortest query, total for measured runs, s", # 6 "", # Runs #7 ] From 4a69bd78819c1bb97988fce96d17be99bbffe00e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 07:37:45 +0200 Subject: [PATCH 35/74] Fix terrible test @arthurpassos --- ...tiple_batches_array_inconsistent_offsets.sh | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh index 83196458a84..c96531ffea9 100755 --- a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-ubsan, no-fasttest +# Tags: long, no-ubsan, no-fasttest, no-parallel, no-asan, no-msan, no-tsan +# This test requires around 10 GB of memory and it is just too much. CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -121,9 +122,12 @@ echo "Parquet" #} DATA_FILE=$CUR_DIR/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (ints Array(Int64), strings Nullable(String)) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" -${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file + +${CLICKHOUSE_LOCAL} --multiquery " +DROP TABLE IF EXISTS parquet_load; +CREATE TABLE parquet_load (ints Array(Int64), strings Nullable(String)) ENGINE = Memory; +INSERT INTO parquet_load FROM INFILE '$DATA_FILE'; +SELECT sum(cityHash64(*)) FROM parquet_load; +SELECT count() FROM parquet_load; +DROP TABLE parquet_load; +" From b8d6c68f5fdc691f18000878203971689d2f812c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Jul 2024 10:30:03 +0200 Subject: [PATCH 36/74] Update 02874_parquet_multiple_batches_array_inconsistent_offsets.reference --- ...arquet_multiple_batches_array_inconsistent_offsets.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference index ba63f2f7e9c..a9eae234dba 100644 --- a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference @@ -1,3 +1,3 @@ Parquet -e76a749f346078a6a43e0cbd25f0d18a - +3249508141921544766 400 From baade8baf45c7919f106160b8b8633d2c59e3ae8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 13:22:50 +0000 Subject: [PATCH 37/74] Replace updateWeakHash to getWeakHash --- src/Columns/ColumnAggregateFunction.cpp | 9 +-- src/Columns/ColumnAggregateFunction.h | 2 +- src/Columns/ColumnArray.cpp | 11 ++- src/Columns/ColumnArray.h | 2 +- src/Columns/ColumnCompressed.h | 3 +- src/Columns/ColumnConst.cpp | 14 +--- src/Columns/ColumnConst.h | 2 +- src/Columns/ColumnDecimal.cpp | 9 +-- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnDynamic.h | 5 +- src/Columns/ColumnFixedString.cpp | 10 +-- src/Columns/ColumnFixedString.h | 2 +- src/Columns/ColumnFunction.h | 5 +- src/Columns/ColumnLowCardinality.cpp | 26 +++--- src/Columns/ColumnLowCardinality.h | 4 +- src/Columns/ColumnMap.cpp | 4 +- src/Columns/ColumnMap.h | 2 +- src/Columns/ColumnNullable.cpp | 16 ++-- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnObject.h | 3 +- src/Columns/ColumnSparse.cpp | 21 ++--- src/Columns/ColumnSparse.h | 2 +- src/Columns/ColumnString.cpp | 9 +-- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnTuple.cpp | 11 ++- src/Columns/ColumnTuple.h | 2 +- src/Columns/ColumnVariant.cpp | 27 ++----- src/Columns/ColumnVariant.h | 2 +- src/Columns/ColumnVector.cpp | 9 +-- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.h | 4 +- src/Columns/IColumnDummy.h | 4 +- src/Columns/IColumnUnique.h | 5 +- src/Columns/tests/gtest_weak_hash_32.cpp | 81 +++++++------------ src/Common/WeakHash.cpp | 22 +++++ src/Common/WeakHash.h | 5 +- src/Interpreters/ConcurrentHashJoin.cpp | 2 +- src/Interpreters/JoinUtils.cpp | 2 +- .../ScatterByPartitionTransform.cpp | 2 +- 39 files changed, 150 insertions(+), 197 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index cfd07c27765..33bd1266c90 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -366,13 +366,10 @@ void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) cons hash.update(wbuf.str().c_str(), wbuf.str().size()); } -void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnAggregateFunction::getWeakHash32() const { auto s = data.size(); - if (hash.getData().size() != data.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), hash.getData().size()); - + WeakHash32 hash(s); auto & hash_data = hash.getData(); std::vector v; @@ -383,6 +380,8 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const wbuf.finalize(); hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]); } + + return hash; } void ColumnAggregateFunction::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 1be7a862438..330a707b75c 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -177,7 +177,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 5d7350f3a79..9203fb8042f 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -271,15 +271,12 @@ void ColumnArray::updateHashWithValue(size_t n, SipHash & hash) const getData().updateHashWithValue(offset + i, hash); } -void ColumnArray::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnArray::getWeakHash32() const { auto s = offsets->size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", s, hash.getData().size()); + WeakHash32 hash(s); - WeakHash32 internal_hash(data->size()); - data->updateWeakHash32(internal_hash); + WeakHash32 internal_hash = data->getWeakHash32(); Offset prev_offset = 0; const auto & offsets_data = getOffsets(); @@ -300,6 +297,8 @@ void ColumnArray::updateWeakHash32(WeakHash32 & hash) const prev_offset = offsets_data[i]; } + + return hash; } void ColumnArray::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 6cd3e2f6c3b..5e01b9144d7 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -82,7 +82,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 5e455709fec..19470113394 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -98,7 +99,7 @@ public: const char * deserializeAndInsertFromArena(const char *) override { throwMustBeDecompressed(); } const char * skipSerializedInArena(const char *) const override { throwMustBeDecompressed(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeDecompressed(); } - void updateWeakHash32(WeakHash32 &) const override { throwMustBeDecompressed(); } + WeakHash32 getWeakHash32() const override { throwMustBeDecompressed(); } void updateHashFast(SipHash &) const override { throwMustBeDecompressed(); } ColumnPtr filter(const Filter &, ssize_t) const override { throwMustBeDecompressed(); } void expand(const Filter &, bool) override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index f2cea83db0e..84427e7be2b 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -137,18 +137,10 @@ void ColumnConst::updatePermutation(PermutationSortDirection /*direction*/, Perm { } -void ColumnConst::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnConst::getWeakHash32() const { - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - WeakHash32 element_hash(1); - data->updateWeakHash32(element_hash); - size_t data_hash = element_hash.getData()[0]; - - for (auto & value : hash.getData()) - value = static_cast(intHashCRC32(data_hash, value)); + WeakHash32 element_hash = data->getWeakHash32(); + return WeakHash32(s, element_hash.getData()[0]); } void ColumnConst::compareColumn( diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index b55a1f42037..65ce53687b9 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -204,7 +204,7 @@ public: data->updateHashWithValue(0, hash); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override { diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index cf413f790a7..ed9c699a841 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -76,13 +76,10 @@ void ColumnDecimal::updateHashWithValue(size_t n, SipHash & hash) const } template -void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnDecimal::getWeakHash32() const { auto s = data.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const T * begin = data.data(); const T * end = begin + s; @@ -94,6 +91,8 @@ void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const ++begin; ++hash_data; } + + return hash; } template diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 32efeb643a6..eb8a305a822 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -102,7 +102,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 9abddc7a26d..6f09abb945a 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -174,9 +175,9 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override + WeakHash32 getWeakHash32() const override { - variant_column->updateWeakHash32(hash); + return variant_column->getWeakHash32(); } void updateHashFast(SipHash & hash) const override diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 1c2de203a94..4d17eb0bebd 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -137,14 +137,10 @@ void ColumnFixedString::updateHashWithValue(size_t index, SipHash & hash) const hash.update(reinterpret_cast(&chars[n * index]), n); } -void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnFixedString::getWeakHash32() const { auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, " - "hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const UInt8 * pos = chars.data(); UInt32 * hash_data = hash.getData().data(); @@ -156,6 +152,8 @@ void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const pos += n; ++hash_data; } + + return hash; } void ColumnFixedString::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 6e88136fc50..630c6c1c0a6 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -133,7 +133,7 @@ public: void updateHashWithValue(size_t index, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index ba924c49a82..dcd67aecad7 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -130,9 +131,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "updateHashWithValue is not implemented for {}", getName()); } - void updateWeakHash32(WeakHash32 &) const override + WeakHash32 getWeakHash32() const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "updateWeakHash32 is not implemented for {}", getName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getWeakHash32 is not implemented for {}", getName()); } void updateHashFast(SipHash &) const override diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index eb694a10b0f..49ce948bf78 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -7,8 +7,7 @@ #include #include #include -#include "Storages/IndicesDescription.h" -#include "base/types.h" +#include #include #include @@ -320,19 +319,10 @@ const char * ColumnLowCardinality::skipSerializedInArena(const char * pos) const return getDictionary().skipSerializedInArena(pos); } -void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnLowCardinality::getWeakHash32() const { - auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - const auto & dict = getDictionary().getNestedColumn(); - WeakHash32 dict_hash(dict->size()); - dict->updateWeakHash32(dict_hash); - - idx.updateWeakHash(hash, dict_hash); + WeakHash32 dict_hash = getDictionary().getNestedColumn()->getWeakHash32(); + return idx.getWeakHash(dict_hash); } void ColumnLowCardinality::updateHashFast(SipHash & hash) const @@ -832,10 +822,11 @@ bool ColumnLowCardinality::Index::containsDefault() const return contains; } -void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const +WeakHash32 ColumnLowCardinality::Index::getWeakHash(const WeakHash32 & dict_hash) const { + WeakHash32 hash(positions->size()); auto & hash_data = hash.getData(); - auto & dict_hash_data = dict_hash.getData(); + const auto & dict_hash_data = dict_hash.getData(); auto update_weak_hash = [&](auto x) { @@ -844,10 +835,11 @@ void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & auto size = data.size(); for (size_t i = 0; i < size; ++i) - hash_data[i] = static_cast(intHashCRC32(dict_hash_data[data[i]], hash_data[i])); + hash_data[i] = dict_hash_data[data[i]]; }; callForType(std::move(update_weak_hash), size_of_type); + return hash; } void ColumnLowCardinality::Index::collectSerializedValueSizes( diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e99be07cd8d..fb0c1237fcf 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -111,7 +111,7 @@ public: getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash &) const override; @@ -325,7 +325,7 @@ public: bool containsDefault() const; - void updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const; + WeakHash32 getWeakHash(const WeakHash32 & dict_hash) const; void collectSerializedValueSizes(PaddedPODArray & sizes, const PaddedPODArray & dict_sizes) const; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 2dffddb2dc9..08d7734ac6b 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -143,9 +143,9 @@ void ColumnMap::updateHashWithValue(size_t n, SipHash & hash) const nested->updateHashWithValue(n, hash); } -void ColumnMap::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnMap::getWeakHash32() const { - nested->updateWeakHash32(hash); + return nested->getWeakHash32(); } void ColumnMap::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index a54071a2974..29275e1b5f7 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -64,7 +64,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index f060e74b315..64e99a3bbe8 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -56,25 +56,21 @@ void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const getNestedColumn().updateHashWithValue(n, hash); } -void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnNullable::getWeakHash32() const { auto s = size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - WeakHash32 old_hash = hash; - nested_column->updateWeakHash32(hash); + WeakHash32 hash = nested_column->getWeakHash32(); const auto & null_map_data = getNullMapData(); auto & hash_data = hash.getData(); - auto & old_hash_data = old_hash.getData(); - /// Use old data for nulls. + /// Use defualt for nulls. for (size_t row = 0; row < s; ++row) if (null_map_data[row]) - hash_data[row] = old_hash_data[row]; + hash_data[row] = WeakHash32::kDefaultInitialValue; + + return hash; } void ColumnNullable::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index a6d0483e527..15bbd8c3b57 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -133,7 +133,7 @@ public: void protect() override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; // Special function for nullable minmax index diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index fadf2e18779..21607e003f2 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -252,7 +253,7 @@ public: const char * deserializeAndInsertFromArena(const char *) override { throwMustBeConcrete(); } const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } - void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } + WeakHash32 getWeakHash32() const override { throwMustBeConcrete(); } void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 98a66e87387..0d103a263dd 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -678,26 +678,13 @@ void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const values->updateHashWithValue(getValueIndex(n), hash); } -void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnSparse::getWeakHash32() const { - if (hash.getData().size() != _size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", _size, hash.getData().size()); - - size_t values_size = values->size(); - WeakHash32 values_hash(values_size); + WeakHash32 values_hash = values->getWeakHash32(); + WeakHash32 hash(size()); auto & hash_data = hash.getData(); auto & values_hash_data = values_hash.getData(); - const auto & offsets_data = getOffsetsData(); - - if (getNumberOfDefaultRows() > 0) - values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; - - for (size_t i = 0; i + 1 < values_size; ++i) - values_hash_data[i + 1] = hash_data[offsets_data[i]]; - - values->updateWeakHash32(values_hash); auto offset_it = begin(); for (size_t i = 0; i < _size; ++i, ++offset_it) @@ -705,6 +692,8 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const size_t value_index = offset_it.getValueIndex(); hash_data[i] = values_hash_data[value_index]; } + + return hash; } void ColumnSparse::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 4860f5171f7..a5d4d788b17 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -139,7 +139,7 @@ public: void protect() override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 1eda9714d62..4accfbe8f41 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -108,13 +108,10 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const return res; } -void ColumnString::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnString::getWeakHash32() const { auto s = offsets.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const UInt8 * pos = chars.data(); UInt32 * hash_data = hash.getData().data(); @@ -130,6 +127,8 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const prev_offset = offset; ++hash_data; } + + return hash; } diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 602ffac65e8..faaaa8848ca 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -212,7 +212,7 @@ public: hash.update(reinterpret_cast(&chars[offset]), string_size); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 9b822d7f570..cb0b05d2154 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -308,16 +308,15 @@ void ColumnTuple::updateHashWithValue(size_t n, SipHash & hash) const column->updateHashWithValue(n, hash); } -void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnTuple::getWeakHash32() const { auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); for (const auto & column : columns) - column->updateWeakHash32(hash); + hash.update(column->getWeakHash32()); + + return hash; } void ColumnTuple::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 38e479791d4..2fafd93f776 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -81,7 +81,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index ee5de4c2dde..8fd6e1bbac1 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -789,36 +789,26 @@ void ColumnVariant::updateHashWithValue(size_t n, SipHash & hash) const variants[localDiscriminatorByGlobal(global_discr)]->updateHashWithValue(offsetAt(n), hash); } -void ColumnVariant::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnVariant::getWeakHash32() const { auto s = size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - /// If we have only NULLs, keep hash unchanged. if (hasOnlyNulls()) - return; + return WeakHash32(s); /// Optimization for case when there is only 1 non-empty variant and no NULLs. /// In this case we can just calculate weak hash for this variant. if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) - { - variants[*non_empty_local_discr]->updateWeakHash32(hash); - return; - } + return variants[*non_empty_local_discr]->getWeakHash32(); /// Calculate weak hash for all variants. std::vector nested_hashes; for (const auto & variant : variants) - { - WeakHash32 nested_hash(variant->size()); - variant->updateWeakHash32(nested_hash); - nested_hashes.emplace_back(std::move(nested_hash)); - } + nested_hashes.emplace_back(variant->getWeakHash32()); /// For each row hash is a hash of corresponding row from corresponding variant. + WeakHash32 hash(s); auto & hash_data = hash.getData(); const auto & local_discriminators_data = getLocalDiscriminators(); const auto & offsets_data = getOffsets(); @@ -827,11 +817,10 @@ void ColumnVariant::updateWeakHash32(WeakHash32 & hash) const Discriminator discr = local_discriminators_data[i]; /// Update hash only for non-NULL values if (discr != NULL_DISCRIMINATOR) - { - auto nested_hash = nested_hashes[local_discriminators_data[i]].getData()[offsets_data[i]]; - hash_data[i] = static_cast(hashCRC32(nested_hash, hash_data[i])); - } + hash_data[i] = nested_hashes[discr].getData()[offsets_data[i]]; } + + return hash; } void ColumnVariant::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index d91b8e93a7d..94f3066e676 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -213,7 +213,7 @@ public: const char * deserializeVariantAndInsertFromArena(Discriminator global_discr, const char * pos); const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 19d1b800961..185a1e0f615 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -73,13 +73,10 @@ void ColumnVector::updateHashWithValue(size_t n, SipHash & hash) const } template -void ColumnVector::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnVector::getWeakHash32() const { auto s = data.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const T * begin = data.data(); const T * end = begin + s; @@ -91,6 +88,8 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const ++begin; ++hash_data; } + + return hash; } template diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 3a0acf5e312..c01778ecf32 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -114,7 +114,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 4b6f34e5aa2..3798d3b7466 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -300,10 +300,10 @@ public: /// passed bytes to hash must identify sequence of values unambiguously. virtual void updateHashWithValue(size_t n, SipHash & hash) const = 0; - /// Update hash function value. Hash is calculated for each element. + /// Get hash function value. Hash is calculated for each element. /// It's a fast weak hash function. Mainly need to scatter data between threads. /// WeakHash32 must have the same size as column. - virtual void updateWeakHash32(WeakHash32 & hash) const = 0; + virtual WeakHash32 getWeakHash32() const = 0; /// Update state of hash with all column. virtual void updateHashFast(SipHash & hash) const = 0; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index c19fb704d9b..b18f4fdb302 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -63,8 +64,9 @@ public: { } - void updateWeakHash32(WeakHash32 & /*hash*/) const override + WeakHash32 getWeakHash32() const override { + return WeakHash32(s); } void updateHashFast(SipHash & /*hash*/) const override diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index 3398452b7ee..1b86204f5b1 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -166,9 +167,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method scatter is not supported for ColumnUnique."); } - void updateWeakHash32(WeakHash32 &) const override + WeakHash32 getWeakHash32() const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method updateWeakHash32 is not supported for ColumnUnique."); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getWeakHash32 is not supported for ColumnUnique."); } void updateHashFast(SipHash &) const override diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index 2c95998761b..3143d0ff83c 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -60,8 +60,7 @@ TEST(WeakHash32, ColumnVectorU8) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -77,8 +76,7 @@ TEST(WeakHash32, ColumnVectorI8) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -94,8 +92,7 @@ TEST(WeakHash32, ColumnVectorU16) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -111,8 +108,7 @@ TEST(WeakHash32, ColumnVectorI16) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -128,8 +124,7 @@ TEST(WeakHash32, ColumnVectorU32) data.push_back(i << 16u); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -145,8 +140,7 @@ TEST(WeakHash32, ColumnVectorI32) data.push_back(i << 16); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -162,8 +156,7 @@ TEST(WeakHash32, ColumnVectorU64) data.push_back(i << 32u); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -179,8 +172,7 @@ TEST(WeakHash32, ColumnVectorI64) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -204,8 +196,7 @@ TEST(WeakHash32, ColumnVectorU128) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -221,8 +212,7 @@ TEST(WeakHash32, ColumnVectorI128) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -238,8 +228,7 @@ TEST(WeakHash32, ColumnDecimal32) data.push_back(i << 16); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -255,8 +244,7 @@ TEST(WeakHash32, ColumnDecimal64) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -272,8 +260,7 @@ TEST(WeakHash32, ColumnDecimal128) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -294,8 +281,7 @@ TEST(WeakHash32, ColumnString1) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -331,8 +317,7 @@ TEST(WeakHash32, ColumnString2) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -369,8 +354,7 @@ TEST(WeakHash32, ColumnString3) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -397,8 +381,7 @@ TEST(WeakHash32, ColumnFixedString) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -444,8 +427,7 @@ TEST(WeakHash32, ColumnArray) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); - WeakHash32 hash(col_arr->size()); - col_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -479,8 +461,7 @@ TEST(WeakHash32, ColumnArray2) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); - WeakHash32 hash(col_arr->size()); - col_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -536,8 +517,7 @@ TEST(WeakHash32, ColumnArrayArray) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); auto col_arr_arr = ColumnArray::create(std::move(col_arr), std::move(off2)); - WeakHash32 hash(col_arr_arr->size()); - col_arr_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -555,8 +535,7 @@ TEST(WeakHash32, ColumnConst) auto col_const = ColumnConst::create(std::move(inner_col), 256); - WeakHash32 hash(col_const->size()); - col_const->updateWeakHash32(hash); + WeakHash32 hash = col_const->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -576,8 +555,7 @@ TEST(WeakHash32, ColumnLowcardinality) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -602,8 +580,7 @@ TEST(WeakHash32, ColumnNullable) auto col_null = ColumnNullable::create(std::move(col), std::move(mask)); - WeakHash32 hash(col_null->size()); - col_null->updateWeakHash32(hash); + WeakHash32 hash = col_null->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -633,8 +610,7 @@ TEST(WeakHash32, ColumnTupleUInt64UInt64) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -671,8 +647,7 @@ TEST(WeakHash32, ColumnTupleUInt64String) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -709,8 +684,7 @@ TEST(WeakHash32, ColumnTupleUInt64FixedString) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -756,8 +730,7 @@ TEST(WeakHash32, ColumnTupleUInt64Array) columns.emplace_back(ColumnArray::create(std::move(val), std::move(off))); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq_data); } diff --git a/src/Common/WeakHash.cpp b/src/Common/WeakHash.cpp index 54d973b6296..cb12df84db1 100644 --- a/src/Common/WeakHash.cpp +++ b/src/Common/WeakHash.cpp @@ -1,2 +1,24 @@ #include +#include +#include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void WeakHash32::update(const WeakHash32 & other) +{ + size_t size = data.size(); + if (size != other.data.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match:" + "left size is {}, right size is {}", size, other.data.size()); + + for (size_t i = 0; i < size; ++i) + data[i] = static_cast(intHashCRC32(other.data[i], data[i])); +} + +} diff --git a/src/Common/WeakHash.h b/src/Common/WeakHash.h index b59624e64f2..d4a8d63868c 100644 --- a/src/Common/WeakHash.h +++ b/src/Common/WeakHash.h @@ -11,9 +11,8 @@ namespace DB /// The main purpose why this class needed is to support data initialization. Initially, every bit is 1. class WeakHash32 { - static constexpr UInt32 kDefaultInitialValue = ~UInt32(0); - public: + static constexpr UInt32 kDefaultInitialValue = ~UInt32(0); using Container = PaddedPODArray; @@ -22,6 +21,8 @@ public: void reset(size_t size, UInt32 initial_value = kDefaultInitialValue) { data.assign(size, initial_value); } + void update(const WeakHash32 & other); + const Container & getData() const { return data; } Container & getData() { return data; } diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 4493a9f4dbd..ac940c62a1a 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -310,7 +310,7 @@ IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_co { const auto & key_col = from_block.getByName(key_name).column->convertToFullColumnIfConst(); const auto & key_col_no_lc = recursiveRemoveLowCardinality(recursiveRemoveSparse(key_col)); - key_col_no_lc->updateWeakHash32(hash); + hash.update(key_col_no_lc->getWeakHash32()); } return hashToSelector(hash, num_shards); } diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 1788c9aca48..180a45d4295 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -554,7 +554,7 @@ static Blocks scatterBlockByHashImpl(const Strings & key_columns_names, const Bl for (const auto & key_name : key_columns_names) { ColumnPtr key_col = materializeColumn(block, key_name); - key_col->updateWeakHash32(hash); + hash.update(key_col->getWeakHash32()); } auto selector = hashToSelector(hash, sharder); diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp index 6e3cdc0fda1..16d265c9bcb 100644 --- a/src/Processors/Transforms/ScatterByPartitionTransform.cpp +++ b/src/Processors/Transforms/ScatterByPartitionTransform.cpp @@ -109,7 +109,7 @@ void ScatterByPartitionTransform::generateOutputChunks() hash.reset(num_rows); for (const auto & column_number : key_columns) - columns[column_number]->updateWeakHash32(hash); + hash.update(columns[column_number]->getWeakHash32()); const auto & hash_data = hash.getData(); IColumn::Selector selector(num_rows); From 05874d0b85d78c1067c1db5332c7cc74b94d88cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 15:37:55 +0000 Subject: [PATCH 38/74] Fixing style. --- src/Columns/ColumnDecimal.cpp | 1 - src/Columns/ColumnNullable.cpp | 2 +- src/Columns/ColumnSparse.cpp | 18 ------------------ src/Columns/ColumnSparse.h | 3 --- 4 files changed, 1 insertion(+), 23 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index ed9c699a841..e27807950ae 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; } template diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 64e99a3bbe8..867c9149242 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -65,7 +65,7 @@ WeakHash32 ColumnNullable::getWeakHash32() const const auto & null_map_data = getNullMapData(); auto & hash_data = hash.getData(); - /// Use defualt for nulls. + /// Use default for nulls. for (size_t row = 0; row < s; ++row) if (null_map_data[row]) hash_data[row] = WeakHash32::kDefaultInitialValue; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0d103a263dd..8f98a4433d3 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -809,24 +809,6 @@ size_t ColumnSparse::getValueIndex(size_t n) const return it - offsets_data.begin() + 1; } -size_t ColumnSparse::getFirstDefaultValueIndex() const -{ - if (getNumberOfDefaultRows() == 0) - return size(); - - const auto & offsets_data = getOffsetsData(); - size_t off_size = offsets_data.size(); - - if (off_size == 0 || offsets_data[0] > 0) - return 0; - - size_t idx = 0; - while (idx + 1 < off_size && offsets_data[idx] + 1 == offsets_data[idx + 1]) - ++idx; - - return offsets_data[idx] + 1; -} - ColumnSparse::Iterator ColumnSparse::getIterator(size_t n) const { const auto & offsets_data = getOffsetsData(); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index a5d4d788b17..392a6910956 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -173,9 +173,6 @@ public: /// O(log(offsets.size())) complexity, size_t getValueIndex(size_t n) const; - /// Returns an index of the first default value, or size() if there is no defaults. - size_t getFirstDefaultValueIndex() const; - const IColumn & getValuesColumn() const { return *values; } IColumn & getValuesColumn() { return *values; } From 3985a4012d70762480d50083eb0d30a3b70e0981 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 00:06:25 +0200 Subject: [PATCH 39/74] Fix tidy --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 4e62c503d65..4a4deb07eee 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -155,7 +155,7 @@ void printExceptionWithRespectToAbort(LoggerPtr log, const String & query_id) { std::rethrow_exception(ex); } - catch (const TestException &) + catch (const TestException &) // NOLINT { /// Exception from a unit test, ignore it. } From 87fafaa9f5f8406228440631acf597c782a3ecdd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 11:00:50 +0000 Subject: [PATCH 40/74] Remove flaky case from the test. --- ..._finctions_and_column_sparse_bug.reference | 73 ------------------- ...window_finctions_and_column_sparse_bug.sql | 54 -------------- 2 files changed, 127 deletions(-) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference index 356329a392d..13e229432ae 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference @@ -8,76 +8,3 @@ true 1 1 true 1 2 --- -755809149 0 ---- -1 -2081147898 -1 -1981899149 -2 -2051538534 -2 -1650266905 -3 -1975508531 -3 -1646738223 -4 -1700730666 -4 -1618912877 -5 -1465484835 -5 -1317193174 -6 -1458338029 -6 -1219769753 -7 -1450619195 -7 -1154269118 -8 -1365934326 -8 -1150980622 -9 -1203382363 -9 -1098155311 -10 -1197430632 -10 -841067875 -11 -1176267855 -11 -816935497 -12 -1020892864 -12 -599948807 -13 -991301833 -13 -526570556 -14 -685902265 -14 -504713125 -15 -653505826 -15 -411038390 -16 -451392958 -16 -331834394 -17 -262516786 -17 -176934810 -18 -222873194 -18 -2 -19 -153185515 -19 6 -20 -74234560 -20 255 -21 -41 -21 406615258 -22 -6 -22 541758331 -23 -5 -23 720985423 -24 -3 -24 745669725 -25 15 -25 897064234 -26 65535 -26 1116921321 -27 77089559 -27 1207796283 -28 100663045 -28 1603772265 -29 561061873 -29 1664059402 -30 643897141 -30 1688303275 -31 914629990 -31 1913361922 -32 1159852204 -32 1929066636 -33 1258218855 -33 1968095908 -34 1459407556 -34 2054878592 -35 1936334332 -35 2125075305 -36 1962246186 -37 2030467062 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index 6e326d0a67f..f2391e0d165 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -31,57 +31,3 @@ SELECT * FROM ( SELECT c, min(w) OVER (PARTITION BY s ORDER BY c ASC, s ASC, w ASC) FROM t limit toUInt64(-1)) WHERE c = -755809149; - -SELECT '---'; - -create table t_vkx4cc ( - c_ylzjpt Int32, - c_hqfr9 Bool , - ) engine = MergeTree order by c_ylzjpt; - -system stop merges t_vkx4cc; - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2081147898, coalesce((NOT NOT(cast( (53 < 539704722) as Nullable(Bool)))), true)), (-1219769753, coalesce((true) and (false), false)), (-1981899149, coalesce(false, false)), (-1646738223, coalesce((NOT NOT(cast( (23.5 <= -26) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (255, coalesce(false, false)), (-1317193174, coalesce(false, false)), (-41, coalesce(true, false)), (1929066636, coalesce(false, true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-1700730666, coalesce((NOT NOT(cast( (-2022515471055597472 AND -29) as Nullable(Bool)))), false)), (1664059402, coalesce((NOT NOT(cast( (-19643 >= -122) as Nullable(Bool)))), false)), (1688303275, coalesce((NOT NOT(cast( (737275892 < 105) as Nullable(Bool)))), true)), (406615258, coalesce((NOT NOT(cast( (-657730213 = 82.86) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2, coalesce(false, false)), (1962246186, coalesce(true, false)), (-991301833, coalesce(true, true)), (2054878592, coalesce(false, false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (643897141, coalesce((NOT NOT(cast( (-60 AND cast(null as Nullable(Int64))) as Nullable(Bool)))), true)), (-2051538534, coalesce(((-1616816511 between 332225780 and -1883087387)) or ((-573375170 between -1427445977 and 615586748)), false)), (77089559, coalesce((NOT NOT(cast( ((true) and (true) != 925456787) as Nullable(Bool)))), false)), (1116921321, coalesce((0 is NULL), true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-816935497, coalesce(false, false)), (1207796283, coalesce((-129 between -5 and -5), false)), (-1365934326, coalesce(true, false)), (-1618912877, coalesce((NOT NOT(cast( (false >= 31833) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-331834394, coalesce((NOT NOT(cast( (-63 <= -1822810052) as Nullable(Bool)))), true)), (-1020892864, coalesce((NOT NOT(cast( (40.31 <= 8146037365746019777) as Nullable(Bool)))), true)), (-1150980622, coalesce(((94019304 between -730556489 and 32)) and ((-956354236 is not NULL)), true)), (-1203382363, coalesce(true, true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-653505826, coalesce((true) or (true), false)), (-1975508531, coalesce(((-796885845 between 65536 and cast(null as Nullable(Int32)))) or ((NOT NOT(cast( (-7467729336434250795 < 100.20) as Nullable(Bool))))), false)), (-1465484835, coalesce(((NOT NOT(cast( (19209 <= 75.96) as Nullable(Bool))))) or (true), false)), (1968095908, coalesce((NOT NOT(cast( (-1309960412156062327 > 13102) as Nullable(Bool)))), true)); - -alter table t_vkx4cc add column c_zosphq2t1 Float64; - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-153185515, coalesce((NOT NOT(cast( (1291639145 >= 30.22) as Nullable(Bool)))), false), -1.8), (-411038390, coalesce(((-762326135 between 16 and 177530758)) or (false), true), 26.34), (914629990, coalesce((-1125832977 is not NULL), true), 59.2), (541758331, coalesce(false, true), -255.1); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (2125075305, coalesce(false, false), 55.36), (-1176267855, coalesce(true, true), 55.45), (1459407556, coalesce((true) and ((NOT NOT(cast( (95.96 != 65) as Nullable(Bool))))), true), 85.80), (-1098155311, coalesce(false, false), 2147483649.9); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (6, coalesce((NOT NOT(cast( (1546334968 < -4) as Nullable(Bool)))), true), 57.42), (-5, coalesce((NOT NOT(cast( (59 AND 13) as Nullable(Bool)))), false), 65536.3), (100663045, coalesce((-1190355242 is not NULL), true), 73.80), (-451392958, coalesce((NOT NOT(cast( (false != -443845933) as Nullable(Bool)))), false), -4294967294.0); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (561061873, coalesce(true, false), 12.17), (-526570556, coalesce(false, false), 64.73), (-1450619195, coalesce(true, true), 54.33), (-3, coalesce(true, true), 52.9); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-504713125, coalesce(false, true), 27.58), (897064234, coalesce((836516994 between cast(null as Nullable(Int32)) and -1832647080), true), 9223372036854775809.2), (65535, coalesce(true, true), 4294967297.5), (-599948807, coalesce((false) or ((NOT NOT(cast( (6.52 = 65.49) as Nullable(Bool))))), false), 256.5); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1650266905, coalesce((NOT NOT(cast( (-83 = -218055084) as Nullable(Bool)))), true), 1.9), (-841067875, coalesce(false, true), -126.5), (15, coalesce(((NOT NOT(cast( (cast(null as Nullable(Decimal)) = cast(null as Nullable(Int32))) as Nullable(Bool))))) or (true), true), 33.65), (1913361922, coalesce((NOT NOT(cast( (false AND 0) as Nullable(Bool)))), false), 6.4); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1159852204, coalesce((-2057115045 is not NULL), false), 20.61), (-6, coalesce(true, true), 66.33), (-1154269118, coalesce(false, true), 8.89), (1258218855, coalesce(true, false), 19.80); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1603772265, coalesce(false, true), 57.87), (-176934810, coalesce(false, true), 128.8), (-1458338029, coalesce((NOT NOT(cast( (20908 != (NOT NOT(cast( (cast(null as Nullable(Decimal)) <= (true) or ((NOT NOT(cast( (973511022 <= -112) as Nullable(Bool)))))) as Nullable(Bool))))) as Nullable(Bool)))), true), 76.54), (-262516786, coalesce((cast(null as Nullable(Int32)) is NULL), false), 21.49); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1197430632, coalesce(true, false), 45.40), (-685902265, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) < cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 5.55), (1936334332, coalesce((-1565552735 is not NULL), false), 26.28), (2030467062, coalesce((NOT NOT(cast( (127.3 != cast(null as Nullable(Int32))) as Nullable(Bool)))), true), 89.50); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (720985423, coalesce((NOT NOT(cast( (-451448940 = cast(null as Nullable(Decimal))) as Nullable(Bool)))), false), 52.65), (-222873194, coalesce(((-20 between -1419620477 and 1616455043)) or ((25624502 between 1312431316 and 1757361651)), false), 127.2), (745669725, coalesce((NOT NOT(cast( ((NOT NOT(cast( (cast(null as Nullable(UInt64)) <= 42) as Nullable(Bool)))) >= 3233811255032796928) as Nullable(Bool)))), false), 7.74), (-74234560, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) >= cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 19.25); - -SELECT DISTINCT - count(ref_0.c_zosphq2t1) over (partition by ref_0.c_hqfr9 order by ref_0.c_ylzjpt, ref_0.c_hqfr9, ref_0.c_zosphq2t1) as c0, - ref_0.c_ylzjpt as c1 -FROM - t_vkx4cc as ref_0 - order by c0, c1; From 959ac9a768530cb9e2c7b013df37bfee000a1644 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 Jul 2024 13:16:35 +0200 Subject: [PATCH 41/74] ci: dump dmesg in case of OOM Without additional info it is unclear how to tune paralelism or maybe split some modules. Signed-off-by: Azat Khuzhin --- tests/ci/ci.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index cf285f4b97d..c5ad97088aa 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1281,7 +1281,8 @@ def main() -> int: except ValueError: pass if Utils.is_killed_with_oom(): - print("WARNING: OOM while job execution") + print("WARNING: OOM while job execution:") + print(subprocess.run("sudo dmesg -T", check=False)) error = f"Out Of Memory, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" else: error = f"Unknown, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" From 85714e7d7e18a9d91dfe385b658e1ae1ddedb2ea Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Jul 2024 11:31:23 +0000 Subject: [PATCH 42/74] fix asof join on nulls --- .../Transforms/MergeJoinTransform.cpp | 10 ++- .../0_stateless/00976_asof_join_on.reference | 78 ++++++++++++++++++- .../0_stateless/00976_asof_join_on.sql.j2 | 7 +- 3 files changed, 88 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 3b69ddaec06..7bdb3d4f30f 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -257,8 +257,14 @@ JoinKeyRow::JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos) new_col->insertFrom(*col, pos); row.push_back(std::move(new_col)); } - if (const auto * asof_column = cursor.getAsofColumn()) + if (const IColumn * asof_column = cursor.getAsofColumn()) { + if (const auto * nullable_asof_column = checkAndGetColumn(asof_column)) + { + /// We save matched column, and since NULL do not match anything, we can't use it as a key + chassert(!nullable_asof_column->isNullAt(pos)); + asof_column = nullable_asof_column->getNestedColumnPtr().get(); + } auto new_col = asof_column->cloneEmpty(); new_col->insertFrom(*asof_column, pos); row.push_back(std::move(new_col)); @@ -1174,7 +1180,6 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (!cursors[1]->cursor.isValid() && !cursors[1]->fullyCompleted()) return Status(1); - if (auto result = handleAllJoinState()) return std::move(*result); @@ -1183,7 +1188,6 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { - if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) return Status(createBlockWithDefaults(0)); diff --git a/tests/queries/0_stateless/00976_asof_join_on.reference b/tests/queries/0_stateless/00976_asof_join_on.reference index e13745bed9d..80af4287d3c 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.reference +++ b/tests/queries/0_stateless/00976_asof_join_on.reference @@ -1,4 +1,4 @@ -- default - +- default / join_use_nulls = 0 - 1 1 0 0 1 2 1 2 1 3 1 2 @@ -35,7 +35,7 @@ 2 1 2 3 2 2 2 3 1 2 1 2 -- full_sorting_merge - +- full_sorting_merge / join_use_nulls = 0 - 1 1 0 0 1 2 1 2 1 3 1 2 @@ -72,3 +72,77 @@ 2 1 2 3 2 2 2 3 1 2 1 2 +- default / join_use_nulls = 1 - +1 1 \N \N +1 2 1 2 +1 3 1 2 +2 1 \N \N +2 2 \N \N +2 3 2 3 +3 1 \N \N +3 2 \N \N +3 3 \N \N +9 +1 2 1 2 +1 3 1 2 +2 3 2 3 +- +1 1 1 2 +1 2 1 2 +1 3 1 4 +2 1 2 3 +2 2 2 3 +2 3 2 3 +- +1 1 1 2 +1 2 1 2 +1 3 1 4 +2 1 2 3 +2 2 2 3 +2 3 2 3 +- +1 3 1 2 +- +1 1 1 2 +1 2 1 4 +1 3 1 4 +2 1 2 3 +2 2 2 3 +1 2 1 2 +- full_sorting_merge / join_use_nulls = 1 - +1 1 \N \N +1 2 1 2 +1 3 1 2 +2 1 \N \N +2 2 \N \N +2 3 2 3 +3 1 \N \N +3 2 \N \N +3 3 \N \N +9 +1 2 1 2 +1 3 1 2 +2 3 2 3 +- +1 1 1 2 +1 2 1 2 +1 3 1 4 +2 1 2 3 +2 2 2 3 +2 3 2 3 +- +1 1 1 2 +1 2 1 2 +1 3 1 4 +2 1 2 3 +2 2 2 3 +2 3 2 3 +- +1 3 1 2 +- +1 1 1 2 +1 2 1 4 +1 3 1 4 +2 1 2 3 +2 2 2 3 +1 2 1 2 diff --git a/tests/queries/0_stateless/00976_asof_join_on.sql.j2 b/tests/queries/0_stateless/00976_asof_join_on.sql.j2 index aecf472a36c..ea642366a05 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.sql.j2 +++ b/tests/queries/0_stateless/00976_asof_join_on.sql.j2 @@ -7,11 +7,13 @@ CREATE TABLE B(b UInt32, t UInt32) ENGINE = Memory; INSERT INTO A (a,t) VALUES (1,1),(1,2),(1,3), (2,1),(2,2),(2,3), (3,1),(3,2),(3,3); INSERT INTO B (b,t) VALUES (1,2),(1,4),(2,3); +{% for join_use_nulls in [0, 1] -%} {% for join_algorithm in ['default', 'full_sorting_merge'] -%} SET join_algorithm = '{{ join_algorithm }}'; -SELECT '- {{ join_algorithm }} -'; +SELECT '- {{ join_algorithm }} / join_use_nulls = {{ join_use_nulls }} -'; +set join_use_nulls = {{ join_use_nulls }}; SELECT A.a, A.t, B.b, B.t FROM A ASOF LEFT JOIN B ON A.a == B.b AND A.t >= B.t ORDER BY (A.a, A.t); SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t; @@ -34,7 +36,8 @@ ASOF INNER JOIN (SELECT * FROM B UNION ALL SELECT 1, 3) AS B ON B.t <= A.t AND A WHERE B.t != 3 ORDER BY (A.a, A.t) ; -{% endfor %} +{% endfor -%} +{% endfor -%} DROP TABLE A; DROP TABLE B; From ab2522b17020f1fb31b760f0594784cd58468797 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Jul 2024 13:17:34 +0000 Subject: [PATCH 43/74] fix asof join on nulls --- src/Processors/Transforms/MergeJoinTransform.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 7bdb3d4f30f..e96a75d277b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -301,7 +301,11 @@ bool JoinKeyRow::asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequalit if (isNullAt(*asof_row, 0) || isNullAt(*cursor.getAsofColumn(), cursor->getRow())) return false; - int cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *asof_row, 1); + int cmp = 0; + if (const auto * nullable_column = checkAndGetColumn(cursor.getAsofColumn())) + cmp = nullable_column->getNestedColumn().compareAt(cursor->getRow(), 0, *asof_row, 1); + else + cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *asof_row, 1); return (asof_inequality == ASOFJoinInequality::Less && cmp < 0) || (asof_inequality == ASOFJoinInequality::LessOrEquals && cmp <= 0) From 093b13329c4206b4ecd604fb373d8d60cf345a1f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Jul 2024 15:46:05 +0000 Subject: [PATCH 44/74] One more check in JOIN ON ... IS NULL --- .../Passes/LogicalExpressionOptimizerPass.cpp | 46 +++++++++++++++++-- ...11_join_on_nullsafe_optimization.reference | 4 ++ .../02911_join_on_nullsafe_optimization.sql | 1 + 3 files changed, 47 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 854697bca9f..7e54b5a4b42 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -67,6 +67,41 @@ QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes) return nullptr; } +/// Checks if the node is combination of isNull and notEquals functions of two the same arguments +bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, QueryTreeNodePtr & rhs) +{ + QueryTreeNodePtrWithHashSet all_arguments; + for (const auto & node : nodes) + { + const auto * func_node = node->as(); + if (!func_node) + return false; + + const auto & arguments = func_node->getArguments().getNodes(); + if (func_node->getFunctionName() == "isNull" && arguments.size() == 1) + all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0])); + else if (func_node->getFunctionName() == "notEquals" && arguments.size() == 2) + { + if (arguments[0]->isEqual(*arguments[1])) + return false; + all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0])); + all_arguments.insert(QueryTreeNodePtrWithHash(arguments[1])); + } + else + return false; + + if (all_arguments.size() > 2) + return false; + } + + if (all_arguments.size() != 2) + return false; + + lhs = all_arguments.begin()->node; + rhs = std::next(all_arguments.begin())->node; + return true; +} + bool isBooleanConstant(const QueryTreeNodePtr & node, bool expected_value) { const auto * constant_node = node->as(); @@ -212,11 +247,14 @@ private: else if (func_name == "and") { const auto & and_arguments = argument_function->getArguments().getNodes(); - bool all_are_is_null = and_arguments.size() == 2 && isNodeFunction(and_arguments[0], "isNull") && isNodeFunction(and_arguments[1], "isNull"); - if (all_are_is_null) + + QueryTreeNodePtr is_null_lhs_arg; + QueryTreeNodePtr is_null_rhs_arg; + if (matchIsNullOfTwoArgs(and_arguments, is_null_lhs_arg, is_null_rhs_arg)) { - is_null_argument_to_indices[getFunctionArgument(and_arguments.front(), 0)].push_back(or_operands.size() - 1); - is_null_argument_to_indices[getFunctionArgument(and_arguments.back(), 0)].push_back(or_operands.size() - 1); + is_null_argument_to_indices[is_null_lhs_arg].push_back(or_operands.size() - 1); + is_null_argument_to_indices[is_null_rhs_arg].push_back(or_operands.size() - 1); + continue; } /// Expression `a = b AND (a IS NOT NULL) AND true AND (b IS NOT NULL)` we can be replaced with `a = b` diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 4eb7e74446d..31a1cda18e7 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -35,6 +35,10 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N 2 2 2 2 3 3 3 33 \N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +2 2 2 2 +3 3 3 33 +\N \N \N \N -- aliases defined in the join condition are valid -- FIXME(@vdimir) broken query formatting for the following queries: -- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index f7813e2a1b4..f739259caf9 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -34,6 +34,7 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) A SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- aliases defined in the join condition are valid -- FIXME(@vdimir) broken query formatting for the following queries: From 69ad57a2a52d50510b4ec5beea08c0c599846859 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 03:58:07 +0200 Subject: [PATCH 45/74] Update 03205_parallel_window_finctions_and_column_sparse_bug.sql --- .../03205_parallel_window_finctions_and_column_sparse_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index f2391e0d165..4cc54bb5ac2 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -6,7 +6,7 @@ insert into t values (1, 0); insert into t values (1, 1); insert into t values (1, 0)(1, 1); -SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2, allow_experimental_analyzer = 1; SELECT '---'; SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; SELECT '---'; From 3fb01ed2c9154d79cd9d23e3ae2e8708e86d0a34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 19 Jul 2024 12:10:28 +0000 Subject: [PATCH 46/74] Use nonexistent address to check connection error at table creation --- tests/integration/test_storage_rabbitmq/test.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3240039ee81..c163f3f7aed 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2220,13 +2220,11 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): def test_rabbitmq_no_connection_at_startup_1(rabbitmq_cluster): - # no connection when table is initialized - rabbitmq_cluster.pause_container("rabbitmq1") - instance.query_and_get_error( + error = instance.query_and_get_error( """ CREATE TABLE test.cs (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = 'no_connection_at_startup:5672', rabbitmq_exchange_name = 'cs', rabbitmq_format = 'JSONEachRow', rabbitmq_flush_interval_ms=1000, @@ -2234,7 +2232,7 @@ def test_rabbitmq_no_connection_at_startup_1(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; """ ) - rabbitmq_cluster.unpause_container("rabbitmq1") + assert "CANNOT_CONNECT_RABBITMQ" in error def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster): From e3b2fbf7ec1a322fa4fcbd808a3a19dd6ec3b8ee Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 15:37:50 +0200 Subject: [PATCH 47/74] CI: POC for Auto Releases --- .github/workflows/auto_release.yml | 42 ++++--- .github/workflows/create_release.yml | 9 +- tests/ci/auto_release.py | 169 +++++++++++++-------------- tests/ci/ci_utils.py | 6 +- 4 files changed, 121 insertions(+), 105 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index f1a6b307b40..e90b183b8d6 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -1,44 +1,58 @@ name: AutoRelease env: - # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 concurrency: - group: auto-release + group: release on: # yamllint disable-line rule:truthy # schedule: # - cron: '0 10-16 * * 1-5' workflow_dispatch: jobs: - CherryPick: - runs-on: [self-hosted, style-checker-aarch64] + AutoRelease: + runs-on: [self-hosted, release-maker] steps: + - name: DebugInfo + uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Set envs - # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings run: | cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/cherry_pick ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] }} + uses: ./.github/workflows/create_release.yml + with: + type: patch + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} + dry-run: true + autorelease: true + - name: Post Slack Message + if: ${{ !cancelled() }} + run: | + echo Slack Message + - name: Clean up run: | docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index d8d27531f28..96cd46f583a 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -22,6 +22,10 @@ concurrency: required: false default: true type: boolean + autorelease: + required: false + default: false + type: boolean jobs: CreateRelease: @@ -30,8 +34,10 @@ jobs: runs-on: [self-hosted, release-maker] steps: - name: DebugInfo + if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Set envs + if: ${{ ! inputs.autorelease }} # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings run: | cat >> "$GITHUB_ENV" << 'EOF' @@ -41,6 +47,7 @@ jobs: RELEASE_INFO_FILE=${{ runner.temp }}/release_info.json EOF - name: Check out repository code + if: ${{ ! inputs.autorelease }} uses: ClickHouse/checkout@v1 with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} @@ -161,6 +168,6 @@ jobs: export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - name: Post Slack Message - if: always() + if: ${{ !cancelled() }} run: | echo Slack Message diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index bfd19b11e6d..88e91eb8aed 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,13 +1,16 @@ import argparse -from datetime import timedelta, datetime +import dataclasses +import json import logging import os -from commit_status_helper import get_commit_filtered_statuses +from typing import List + from get_robot_token import get_best_robot_token from github_helper import GitHub -from release import Release, Repo as ReleaseRepo, RELEASE_READY_STATUS -from report import SUCCESS from ssh import SSHKey +from ci_utils import Shell +from env_helper import GITHUB_REPOSITORY +from report import SUCCESS LOGGER_NAME = __name__ HELPER_LOGGERS = ["github_helper", LOGGER_NAME] @@ -20,116 +23,104 @@ def parse_args(): "branches and do a release in case for green builds." ) parser.add_argument("--token", help="GitHub token, if not set, used from smm") - parser.add_argument( - "--repo", default="ClickHouse/ClickHouse", help="Repo owner/name" - ) - parser.add_argument("--dry-run", action="store_true", help="Do not create anything") - parser.add_argument( - "--release-after-days", - type=int, - default=3, - help="Do automatic release on the latest green commit after the latest " - "release if the newest release is older than the specified days", - ) - parser.add_argument( - "--debug-helpers", - action="store_true", - help="Add debug logging for this script and github_helper", - ) - parser.add_argument( - "--remote-protocol", - "-p", - default="ssh", - choices=ReleaseRepo.VALID, - help="repo protocol for git commands remote, 'origin' is a special case and " - "uses 'origin' as a remote", - ) return parser.parse_args() +MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 +AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" + + +@dataclasses.dataclass +class ReleaseParams: + release_branch: str + commit_sha: str + + +@dataclasses.dataclass +class AutoReleaseInfo: + releases: List[ReleaseParams] + + def add_release(self, release_params: ReleaseParams): + self.releases.append(release_params) + + def dump(self): + print(f"Dump release info into [{AUTORELEASE_INFO_FILE}]") + with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: + print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + + def main(): args = parse_args() - logging.basicConfig(level=logging.INFO) - if args.debug_helpers: - for logger_name in HELPER_LOGGERS: - logging.getLogger(logger_name).setLevel(logging.DEBUG) token = args.token or get_best_robot_token() - days_as_timedelta = timedelta(days=args.release_after_days) - now = datetime.now() - + assert len(token) > 10 + os.environ["GH_TOKEN"] = token + (Shell.run("gh auth status", check=True)) gh = GitHub(token) - prs = gh.get_release_pulls(args.repo) + prs = gh.get_release_pulls(GITHUB_REPOSITORY) branch_names = [pr.head.ref for pr in prs] - logger.info("Found release branches: %s\n ", " \n".join(branch_names)) - repo = gh.get_repo(args.repo) + print(f"Found release branches [{branch_names}]") + repo = gh.get_repo(GITHUB_REPOSITORY) - # In general there is no guarantee on which order the refs/commits are - # returned from the API, so we have to order them. + autoRelease_info = AutoReleaseInfo(releases=[]) for pr in prs: - logger.info("Checking PR %s", pr.head.ref) + print(f"Checking PR [{pr.head.ref}]") refs = list(repo.get_git_matching_refs(f"tags/v{pr.head.ref}")) refs.sort(key=lambda ref: ref.ref) latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - logger.info("That last release was done at %s", latest_release_tag.tagger.date) - - if latest_release_tag.tagger.date + days_as_timedelta > now: - logger.info( - "Not enough days since the last release %s," - " no automatic release can be done", - latest_release_tag.tag, + commit_num = int( + Shell.run( + f"git rev-list --count {latest_release_tag.tag}..origin/{pr.head.ref}", + check=True, ) - continue - - unreleased_commits = list( - repo.get_commits(sha=pr.head.ref, since=latest_release_tag.tagger.date) ) - unreleased_commits.sort( - key=lambda commit: commit.commit.committer.date, reverse=True + print( + f"Previous release is [{latest_release_tag}] was [{commit_num}] commits before, date [{latest_release_tag.tagger.date}]" ) - - for commit in unreleased_commits: - logger.info("Checking statuses of commit %s", commit.sha) - statuses = get_commit_filtered_statuses(commit) - all_success = all(st.state == SUCCESS for st in statuses) - passed_ready_for_release_check = any( - st.context == RELEASE_READY_STATUS and st.state == SUCCESS - for st in statuses + commit_reverse_index = 0 + commit_found = False + commit_checked = False + commit_sha = "" + while ( + commit_reverse_index < commit_num - 1 + and commit_reverse_index < MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE + ): + commit_checked = True + commit_sha = Shell.run( + f"git rev-list --max-count=1 --skip={commit_reverse_index} origin/{pr.head.ref}", + check=True, ) - if not (all_success and passed_ready_for_release_check): - logger.info("Commit is not green, thus not suitable for release") - continue - - logger.info("Commit is ready for release, let's release!") - - release = Release( - ReleaseRepo(args.repo, args.remote_protocol), - commit.sha, - "patch", - args.dry_run, - True, + print( + f"Check if commit [{commit_sha}] [{pr.head.ref}~{commit_reverse_index}] is ready for release" ) - try: - release.do(True, True, True) - except: - if release.has_rollback: - logging.error( - "!!The release process finished with error, read the output carefully!!" - ) - logging.error( - "Probably, rollback finished with error. " - "If you don't see any of the following commands in the output, " - "execute them manually:" - ) - release.log_rollback() - raise - logging.info("New release is done!") + commit_reverse_index += 1 + + cmd = f"gh api -H 'Accept: application/vnd.github.v3+json' /repos/{GITHUB_REPOSITORY}/commits/{commit_sha}/status" + ci_status_json = Shell.run(cmd, check=True) + ci_status = json.loads(ci_status_json)["state"] + if ci_status == SUCCESS: + commit_found = True break + if commit_found: + print( + f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" + ) + autoRelease_info.add_release( + ReleaseParams(release_branch=pr.head.ref, commit_sha=commit_sha) + ) + else: + print(f"WARNING: No good commits found for release branch [{pr.head.ref}]") + if commit_checked: + print( + f"ERROR: CI is failed. check CI status for branch [{pr.head.ref}]" + ) + + autoRelease_info.dump() if __name__ == "__main__": diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 49f0447b5ca..0653374356f 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -95,7 +95,8 @@ class Shell: return res.stdout.strip() @classmethod - def run(cls, command): + def run(cls, command, check=False): + print(f"Run command [{command}]") res = "" result = subprocess.run( command, @@ -107,6 +108,9 @@ class Shell: ) if result.returncode == 0: res = result.stdout + elif check: + print(f"ERROR: stdout {result.stdout}, stderr {result.stderr}") + assert result.returncode == 0 return res.strip() @classmethod From 3de472cedc22c1b109256507d81a491f9cf57d58 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 17:07:49 +0200 Subject: [PATCH 48/74] add buddy, move release into action.yml try callable wf fix add ci buddy --- .github/actions/release/action.yml | 165 ++++++++++++++++++++++++ .github/workflows/auto_release.yml | 55 +++++++- .github/workflows/create_release.yml | 140 +------------------- tests/ci/artifactory.py | 32 ++--- tests/ci/auto_release.py | 183 ++++++++++++++++++--------- tests/ci/ci.py | 2 +- tests/ci/ci_buddy.py | 56 +++++++- tests/ci/ci_config.py | 3 + tests/ci/ci_utils.py | 76 ++++++++++- tests/ci/create_release.py | 106 ++++++++++------ tests/ci/pr_info.py | 13 +- 11 files changed, 560 insertions(+), 271 deletions(-) create mode 100644 .github/actions/release/action.yml diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml new file mode 100644 index 00000000000..fd4436f1f16 --- /dev/null +++ b/.github/actions/release/action.yml @@ -0,0 +1,165 @@ +name: Release + +description: Makes patch releases and creates new release branch + +inputs: + ref: + description: 'Git reference (branch or commit sha) from which to create the release' + required: true + type: string + type: + description: 'The type of release: "new" for a new release or "patch" for a patch release' + required: true + type: choice + options: + - patch + - new + dry-run: + description: 'Dry run' + required: false + default: true + type: boolean + token: + required: true + type: string + +runs: + using: "composite" + steps: + - name: Prepare Release Info + shell: bash + run: | + python3 ./tests/ci/create_release.py --prepare-release-info \ + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ + ${{ inputs.dry-run && '--dry-run' || '' }} + echo "::group::Release Info" + python3 -m json.tool /tmp/release_info.json + echo "::endgroup::" + release_tag=$(jq -r '.release_tag' /tmp/release_info.json) + commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + echo "Release Tag: $release_tag" + echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" + echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + - name: Download All Release Artifacts + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Push Git Tag for the Release + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Push New Release Branch + if: ${{ inputs.type == 'new' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Checkout master + shell: bash + run: | + git checkout master + - name: Bump Docker versions, Changelog, Security + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 + echo "List versions" + ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv + echo "Update docker version" + ./utils/list-versions/update-docker-version.sh + echo "Generate ChangeLog" + export CI=1 + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ + --volume=".:/ClickHouse" clickhouse/style-test \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --gh-user-or-token=${{ inputs.token }} --jobs=5 \ + --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md + echo "Generate Security" + python3 ./utils/security-generator/generate_security.py > SECURITY.md + git diff HEAD + - name: Create ChangeLog PR + if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + uses: peter-evans/create-pull-request@v6 + with: + author: "robot-clickhouse " + token: ${{ inputs.token }} + committer: "robot-clickhouse " + commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + branch: auto/${{ env.RELEASE_TAG }} + assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher + delete-branch: true + title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} + labels: do not test + body: | + Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + ### Changelog category (leave one): + - Not for changelog (changelog entry is not required) + - name: Reset changes if Dry-run + if: ${{ inputs.dry-run }} + shell: bash + run: | + git reset --hard HEAD + - name: Checkout back to GITHUB_REF + shell: bash + run: | + git checkout "$GITHUB_REF_NAME" + - name: Create GH Release + shell: bash + if: ${{ inputs.type == 'patch' }} + run: | + python3 ./tests/ci/create_release.py --create-gh-release \ + ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Docker clickhouse/clickhouse-server building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + export CHECK_NAME="Docker server image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + - name: Docker clickhouse/clickhouse-keeper building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + export CHECK_NAME="Docker keeper image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + - name: Post Slack Message + if: ${{ !cancelled() }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run && '--dry-run' || '' }} diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index e90b183b8d6..e0a94d3bbb1 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -2,6 +2,7 @@ name: AutoRelease env: PYTHONUNBUFFERED: 1 + DRY_RUN: true concurrency: group: release @@ -9,6 +10,12 @@ on: # yamllint disable-line rule:truthy # schedule: # - cron: '0 10-16 * * 1-5' workflow_dispatch: + inputs: + dry-run: + description: 'Dry run' + required: false + default: true + type: boolean jobs: AutoRelease: @@ -31,7 +38,7 @@ jobs: - name: Auto Release Prepare run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py + python3 auto_release.py --prepare echo "::group::Auto Release Info" python3 -m json.tool /tmp/autorelease_info.json echo "::endgroup::" @@ -40,14 +47,50 @@ jobs: cat /tmp/autorelease_info.json echo 'EOF' } >> "$GITHUB_ENV" + - name: Post Release Branch statuses + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-status - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] }} - uses: ./.github/workflows/create_release.yml + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[0].ready }} + uses: ./.github/actions/release with: - type: patch ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} - dry-run: true - autorelease: true + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Post Slack Message if: ${{ !cancelled() }} run: | diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 96cd46f583a..6246306e536 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -3,7 +3,7 @@ name: CreateRelease concurrency: group: release -'on': +on: workflow_dispatch: inputs: ref: @@ -22,10 +22,6 @@ concurrency: required: false default: true type: boolean - autorelease: - required: false - default: false - type: boolean jobs: CreateRelease: @@ -36,138 +32,16 @@ jobs: - name: DebugInfo if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - - name: Set envs - if: ${{ ! inputs.autorelease }} - # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings - run: | - cat >> "$GITHUB_ENV" << 'EOF' - ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" - echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" - - name: Download All Release Artifacts - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --infile "$RELEASE_INFO_FILE" --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Push Git Tag for the Release - run: | - python3 ./tests/ci/create_release.py --push-release-tag --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} - run: | - python3 ./tests/ci/create_release.py --push-new-release-branch --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Checkout master - run: | - git checkout master - - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} - run: | - [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 - echo "List versions" - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - echo "Update docker version" - ./utils/list-versions/update-docker-version.sh - echo "Generate ChangeLog" - export CI=1 - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token="$GH_TOKEN" --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} - git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md - echo "Generate Security" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} - uses: peter-evans/create-pull-request@v6 + - name: Call Release Action + uses: ./.github/actions/release with: - author: "robot-clickhouse " - token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - branch: auto/${{ env.RELEASE_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) - - name: Reset changes if Dry-run - if: ${{ inputs.dry-run }} - run: | - git reset --hard HEAD - - name: Checkout back to GITHUB_REF - run: | - git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release \ - --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - - name: Export TGZ Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test TGZ Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Export RPM Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test RPM Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Export Debian Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test Debian Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Docker clickhouse/clickhouse-server building - if: ${{ inputs.type == 'patch' }} - run: | - cd "./tests/ci" - export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - - name: Docker clickhouse/clickhouse-keeper building - if: ${{ inputs.type == 'patch' }} - run: | - cd "./tests/ci" - export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - - name: Post Slack Message - if: ${{ !cancelled() }} - run: | - echo Slack Message + ref: ${{ inputs.ref }} + type: inputs.type + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 1a062d05a23..2009b122a18 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -4,7 +4,7 @@ from pathlib import Path from typing import Optional from shutil import copy2 from create_release import PackageDownloader, ReleaseInfo, ShellRunner -from ci_utils import WithIter +from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): @@ -141,12 +141,16 @@ class DebianArtifactory: ShellRunner.run("sync") def test_packages(self): - ShellRunner.run("docker pull ubuntu:latest") + Shell.run("docker pull ubuntu:latest") print(f"Test packages installation, version [{self.version}]") - cmd = f"docker run --rm ubuntu:latest bash -c \"apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-client={self.version}\"" + debian_command = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static={self.version} clickhouse-client={self.version}" + cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' print("Running test command:") print(f" {cmd}") - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + release_info = ReleaseInfo.from_file() + release_info.debian_command = debian_command + release_info.dump() def _copy_if_not_exists(src: Path, dst: Path) -> Path: @@ -210,15 +214,19 @@ class RpmArtifactory: pub_key_path.write_text(ShellRunner.run(update_public_key)[1]) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - ShellRunner.run("sync") + Shell.run("sync") def test_packages(self): - ShellRunner.run("docker pull fedora:latest") + Shell.run("docker pull fedora:latest") print(f"Test package installation, version [{self.version}]") - cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1"' + rpm_command = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1" + cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' print("Running test command:") print(f" {cmd}") - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + release_info = ReleaseInfo.from_file() + release_info.rpm_command = rpm_command + release_info.dump() class TgzArtifactory: @@ -280,12 +288,6 @@ def parse_args() -> argparse.Namespace: formatter_class=argparse.ArgumentDefaultsHelpFormatter, description="Adds release packages to the repository", ) - parser.add_argument( - "--infile", - type=str, - required=True, - help="input file with release info", - ) parser.add_argument( "--export-debian", action="store_true", @@ -328,7 +330,7 @@ if __name__ == "__main__": args = parse_args() assert args.dry_run - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() """ Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: ERROR : IO error: NotImplemented: versionId not implemented diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 88e91eb8aed..b8f781c4d73 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,20 +1,17 @@ import argparse import dataclasses import json -import logging import os +import sys from typing import List from get_robot_token import get_best_robot_token from github_helper import GitHub -from ssh import SSHKey from ci_utils import Shell from env_helper import GITHUB_REPOSITORY from report import SUCCESS - -LOGGER_NAME = __name__ -HELPER_LOGGERS = ["github_helper", LOGGER_NAME] -logger = logging.getLogger(LOGGER_NAME) +from ci_buddy import CIBuddy +from ci_config import CI def parse_args(): @@ -23,8 +20,17 @@ def parse_args(): "branches and do a release in case for green builds." ) parser.add_argument("--token", help="GitHub token, if not set, used from smm") - - return parser.parse_args() + parser.add_argument( + "--post-status", + action="store_true", + help="Post release branch statuses", + ) + parser.add_argument( + "--prepare", + action="store_true", + help="Prepare autorelease info", + ) + return parser.parse_args(), parser MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 @@ -33,8 +39,16 @@ AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" @dataclasses.dataclass class ReleaseParams: + ready: bool + ci_status: str + num_patches: int release_branch: str commit_sha: str + commits_to_branch_head: int + latest: bool + + def to_dict(self): + return dataclasses.asdict(self) @dataclasses.dataclass @@ -49,83 +63,128 @@ class AutoReleaseInfo: with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + @staticmethod + def from_file() -> "AutoReleaseInfo": + with open(AUTORELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: + res = json.load(json_file) + releases = [ReleaseParams(**release) for release in res["releases"]] + return AutoReleaseInfo(releases=releases) -def main(): - args = parse_args() - token = args.token or get_best_robot_token() +def _prepare(token): assert len(token) > 10 os.environ["GH_TOKEN"] = token - (Shell.run("gh auth status", check=True)) + Shell.run("gh auth status", check=True) + gh = GitHub(token) prs = gh.get_release_pulls(GITHUB_REPOSITORY) + prs.sort(key=lambda x: x.head.ref) branch_names = [pr.head.ref for pr in prs] - print(f"Found release branches [{branch_names}]") - repo = gh.get_repo(GITHUB_REPOSITORY) + repo = gh.get_repo(GITHUB_REPOSITORY) autoRelease_info = AutoReleaseInfo(releases=[]) + for pr in prs: - print(f"Checking PR [{pr.head.ref}]") + print(f"\nChecking PR [{pr.head.ref}]") refs = list(repo.get_git_matching_refs(f"tags/v{pr.head.ref}")) - refs.sort(key=lambda ref: ref.ref) + assert refs + refs.sort(key=lambda ref: ref.ref) latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - commit_num = int( - Shell.run( - f"git rev-list --count {latest_release_tag.tag}..origin/{pr.head.ref}", - check=True, - ) - ) - print( - f"Previous release is [{latest_release_tag}] was [{commit_num}] commits before, date [{latest_release_tag.tagger.date}]" - ) - commit_reverse_index = 0 - commit_found = False - commit_checked = False - commit_sha = "" - while ( - commit_reverse_index < commit_num - 1 - and commit_reverse_index < MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE - ): - commit_checked = True - commit_sha = Shell.run( - f"git rev-list --max-count=1 --skip={commit_reverse_index} origin/{pr.head.ref}", - check=True, - ) - print( - f"Check if commit [{commit_sha}] [{pr.head.ref}~{commit_reverse_index}] is ready for release" - ) - commit_reverse_index += 1 - cmd = f"gh api -H 'Accept: application/vnd.github.v3+json' /repos/{GITHUB_REPOSITORY}/commits/{commit_sha}/status" - ci_status_json = Shell.run(cmd, check=True) - ci_status = json.loads(ci_status_json)["state"] - if ci_status == SUCCESS: - commit_found = True - break - if commit_found: + commits = Shell.run( + f"git rev-list --first-parent {latest_release_tag.tag}..origin/{pr.head.ref}", + check=True, + ).split("\n") + commit_num = len(commits) + print( + f"Previous release [{latest_release_tag.tag}] was [{commit_num}] commits ago, date [{latest_release_tag.tagger.date}]" + ) + + commits_to_check = commits[:-1] # Exclude the version bump commit + commit_sha = "" + commit_ci_status = "" + commits_to_branch_head = 0 + + for idx, commit in enumerate( + commits_to_check[:MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE] + ): + print( + f"Check commit [{commit}] [{pr.head.ref}~{idx+1}] as release candidate" + ) + commit_num -= 1 + + is_completed = CI.GHActions.check_wf_completed( + token=token, commit_sha=commit + ) + if not is_completed: + print(f"CI is in progress for [{commit}] - check previous commit") + commits_to_branch_head += 1 + continue + + commit_ci_status = CI.GHActions.get_commit_status_by_name( + token=token, + commit_sha=commit, + status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), + ) + commit_sha = commit + if commit_ci_status == SUCCESS: + break + else: + print(f"CI status [{commit_ci_status}] - skip") + commits_to_branch_head += 1 + + ready = commit_ci_status == SUCCESS and commit_sha + if ready: print( f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" ) - autoRelease_info.add_release( - ReleaseParams(release_branch=pr.head.ref, commit_sha=commit_sha) - ) else: - print(f"WARNING: No good commits found for release branch [{pr.head.ref}]") - if commit_checked: - print( - f"ERROR: CI is failed. check CI status for branch [{pr.head.ref}]" - ) + print(f"WARNING: No ready commits found for release branch [{pr.head.ref}]") + + autoRelease_info.add_release( + ReleaseParams( + release_branch=pr.head.ref, + commit_sha=commit_sha, + ready=ready, + ci_status=commit_ci_status, + num_patches=commit_num, + commits_to_branch_head=commits_to_branch_head, + latest=False, + ) + ) + + if autoRelease_info.releases: + autoRelease_info.releases[-1].latest = True autoRelease_info.dump() -if __name__ == "__main__": - if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): - with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): - main() +def main(): + args, parser = parse_args() + + if args.post_status: + info = AutoReleaseInfo.from_file() + for release_info in info.releases: + if release_info.ready: + CIBuddy(dry_run=False).post_info( + title=f"Auto Release Status for {release_info.release_branch}", + body=release_info.to_dict(), + ) + else: + CIBuddy(dry_run=False).post_warning( + title=f"Auto Release Status for {release_info.release_branch}", + body=release_info.to_dict(), + ) + elif args.prepare: + _prepare(token=args.token or get_best_robot_token()) else: - main() + parser.print_help() + sys.exit(2) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 110a7b2a49c..8ad358cb874 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1295,7 +1295,7 @@ def main() -> int: error_description = f"Out Of Memory, exit_code {job_report.exit_code}" else: error_description = f"Unknown, exit_code {job_report.exit_code}" - CIBuddy().post_error( + CIBuddy().post_job_error( error_description + f" after {int(job_report.duration)}s", job_name=_get_ext_check_name(args.job_name), ) diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index c650b876610..727a3d88359 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -1,5 +1,6 @@ import json import os +from typing import Union, Dict import boto3 import requests @@ -60,7 +61,55 @@ class CIBuddy: except Exception as e: print(f"ERROR: Failed to post message, ex {e}") - def post_error(self, error_description, job_name="", with_instance_info=True): + def _post_formatted( + self, title, body: Union[Dict, str], with_wf_link: bool + ) -> None: + message = title + if isinstance(body, dict): + for name, value in body.items(): + if "commit_sha" in name: + value = ( + f"" + ) + message += f" *{name}*: {value}\n" + else: + message += body + "\n" + run_id = os.getenv("GITHUB_RUN_ID", "") + if with_wf_link and run_id: + message += f" *workflow*: \n" + self.post(message) + + def post_info( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":white_circle: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_done( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":white_check_mark: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_warning( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":warning: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_critical( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":black_circle: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_job_error( + self, + error_description, + job_name="", + with_instance_info=True, + with_wf_link: bool = True, + ): instance_id, instance_type = "unknown", "unknown" if with_instance_info: instance_id = Shell.run("ec2metadata --instance-id") or instance_id @@ -82,10 +131,13 @@ class CIBuddy: message += line_pr_ else: message += line_br_ + run_id = os.getenv("GITHUB_RUN_ID", "") + if with_wf_link and run_id: + message += f" *workflow*: \n" self.post(message) if __name__ == "__main__": # test buddy = CIBuddy(dry_run=True) - buddy.post_error("TEst") + buddy.post_job_error("TEst") diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0a4ab3a823b..a44b15f34c1 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -32,6 +32,9 @@ class CI: from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners + from ci_utils import Envs as Envs + from ci_utils import Utils as Utils + from ci_utils import GHActions as GHActions from ci_definitions import Labels as Labels from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 0653374356f..4536d1f2b54 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,9 +1,16 @@ import os import re import subprocess +import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Tuple +from typing import Any, Iterator, List, Union, Optional, Sequence + +import requests + + +class Envs: + GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") LABEL_CATEGORIES = { @@ -80,6 +87,62 @@ class GHActions: print(line) print("::endgroup::") + @staticmethod + def get_commit_status_by_name( + token: str, commit_sha: str, status_name: Union[str, Sequence] + ) -> Optional[str]: + assert len(token) == 40 + assert len(commit_sha) == 40 + assert is_hex(commit_sha) + assert not is_hex(token) + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses?per_page={200}" + headers = { + "Authorization": f"token {token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get(url, headers=headers, timeout=5) + + if isinstance(status_name, str): + status_name = (status_name,) + if response.status_code == 200: + assert "next" not in response.links, "Response truncated" + statuses = response.json() + for status in statuses: + if status["context"] in status_name: + return status["state"] + return None + + @staticmethod + def check_wf_completed(token: str, commit_sha: str) -> bool: + headers = { + "Authorization": f"token {token}", + "Accept": "application/vnd.github.v3+json", + } + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/check-runs?per_page={100}" + + for i in range(3): + try: + response = requests.get(url, headers=headers, timeout=5) + response.raise_for_status() + # assert "next" not in response.links, "Response truncated" + + data = response.json() + assert data["check_runs"], "?" + + for check in data["check_runs"]: + if check["status"] != "completed": + print( + f" Check workflow status: Check not completed [{check['name']}]" + ) + return False + else: + return True + except Exception as e: + print(f"ERROR: exception {e}") + time.sleep(1) + + return False + class Shell: @classmethod @@ -108,15 +171,18 @@ class Shell: ) if result.returncode == 0: res = result.stdout - elif check: - print(f"ERROR: stdout {result.stdout}, stderr {result.stderr}") - assert result.returncode == 0 + else: + print( + f"ERROR: stdout {result.stdout.strip()}, stderr {result.stderr.strip()}" + ) + if check: + assert result.returncode == 0 return res.strip() @classmethod def check(cls, command): result = subprocess.run( - command + " 2>&1", + command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE, diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 277134c3991..414ec8afd3e 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -14,6 +14,7 @@ from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper from ci_utils import Shell +from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -27,6 +28,7 @@ from ci_config import CI CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) +RELEASE_INFO_FILE = "/tmp/release_info.json" class ShellRunner: @@ -67,17 +69,25 @@ class ReleaseInfo: codename: str previous_release_tag: str previous_release_sha: str + changelog_pr: str = "" + version_bump_pr: str = "" + release_url: str = "" + debian_command: str = "" + rpm_command: str = "" @staticmethod - def from_file(file_path: str) -> "ReleaseInfo": - with open(file_path, "r", encoding="utf-8") as json_file: + def from_file() -> "ReleaseInfo": + with open(RELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: res = json.load(json_file) return ReleaseInfo(**res) + def dump(self): + print(f"Dump release info into [{RELEASE_INFO_FILE}]") + with open(RELEASE_INFO_FILE, "w", encoding="utf-8") as f: + print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + @staticmethod - def prepare(commit_ref: str, release_type: str, outfile: str) -> None: - Path(outfile).parent.mkdir(parents=True, exist_ok=True) - Path(outfile).unlink(missing_ok=True) + def prepare(commit_ref: str, release_type: str) -> None: version = None release_branch = None release_tag = None @@ -91,7 +101,7 @@ class ReleaseInfo: f"git merge-base --is-ancestor origin/{commit_ref} origin/master" ) with checkout(commit_ref): - _, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}") + commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -112,7 +122,7 @@ class ReleaseInfo: assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - _, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}") + commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -171,8 +181,7 @@ class ReleaseInfo: previous_release_tag=previous_release_tag, previous_release_sha=previous_release_sha, ) - with open(outfile, "w", encoding="utf-8") as f: - print(json.dumps(dataclasses.asdict(res), indent=2), file=f) + res.dump() def push_release_tag(self, dry_run: bool) -> None: if dry_run: @@ -276,21 +285,38 @@ class ReleaseInfo: f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" ) + def update_release_info(self, dry_run: bool) -> None: + branch = f"auto/{release_info.release_tag}" + if not dry_run: + get_url_cmd = f"gh pr list --repo {GITHUB_REPOSITORY} --head {branch} --json url --jq '.[0].url'" + url = Shell.run(get_url_cmd) + if url: + print(f"Update release info with Changelog PR link [{url}]") + else: + print(f"WARNING: Changelog PR not found, branch [{branch}]") + else: + url = "dry-run" + + self.changelog_pr = url + self.dump() + def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: repo = os.getenv("GITHUB_REPOSITORY") assert repo - cmds = [] - cmds.append( + cmds = [ f"gh release create --repo {repo} --title 'Release {self.release_tag}' {self.release_tag}" - ) + ] for file in packages_files: cmds.append(f"gh release upload {self.release_tag} {file}") if not dry_run: for cmd in cmds: - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" else: print("Dry-run, would run commands:") print("\n * ".join(cmds)) + self.release_url = f"dry-run" + self.dump() class RepoTypes: @@ -508,6 +534,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Create GH Release object and attach all packages", ) + parser.add_argument( + "--post-status", + action="store_true", + help="Post release status into Slack", + ) parser.add_argument( "--ref", type=str, @@ -525,18 +556,6 @@ def parse_args() -> argparse.Namespace: action="store_true", help="do not make any actual changes in the repo, just show what will be done", ) - parser.add_argument( - "--outfile", - default="", - type=str, - help="output file to write json result to, if not set - stdout", - ) - parser.add_argument( - "--infile", - default="", - type=str, - help="input file with release info", - ) return parser.parse_args() @@ -547,7 +566,7 @@ def checkout(ref: str) -> Iterator[None]: rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref if ref not in (orig_ref,): - ShellRunner.run(f"{GIT_PREFIX} checkout {ref}") + Shell.run(f"{GIT_PREFIX} checkout {ref}") try: yield except (Exception, KeyboardInterrupt) as e: @@ -587,27 +606,21 @@ if __name__ == "__main__": if args.prepare_release_info: assert ( - args.ref and args.release_type and args.outfile - ), "--ref, --release-type and --outfile must be provided with --prepare-release-info" - ReleaseInfo.prepare( - commit_ref=args.ref, release_type=args.release_type, outfile=args.outfile - ) + args.ref and args.release_type + ), "--ref and --release-type must be provided with --prepare-release-info" + ReleaseInfo.prepare(commit_ref=args.ref, release_type=args.release_type) if args.push_release_tag: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.push_release_tag(dry_run=args.dry_run) if args.push_new_release_branch: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.push_new_release_branch(dry_run=args.dry_run) if args.create_bump_version_pr: # TODO: store link to PR in release info - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.update_version_and_contributors_list(dry_run=args.dry_run) if args.download_packages: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() p = PackageDownloader( release=release_info.release_branch, commit_sha=release_info.commit_sha, @@ -615,14 +628,23 @@ if __name__ == "__main__": ) p.run() if args.create_gh_release: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() p = PackageDownloader( release=release_info.release_branch, commit_sha=release_info.commit_sha, version=release_info.version, ) - release_info.create_gh_release(p.get_all_packages_files(), args.dry_run) + if args.post_status: + release_info = ReleaseInfo.from_file() + release_info.update_release_info(dry_run=args.dry_run) + if release_info.debian_command: + CIBuddy(dry_run=args.dry_run).post_done( + f"New release issued", dataclasses.asdict(release_info) + ) + else: + CIBuddy(dry_run=args.dry_run).post_critical( + f"Failed to issue new release", dataclasses.asdict(release_info) + ) # tear down ssh if _ssh_agent and _key_pub: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 9f3b5a586cc..59806a2a8fa 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -296,13 +296,16 @@ class PRInfo: else: if "schedule" in github_event: self.event_type = EventType.SCHEDULE - else: + elif "inputs" in github_event: # assume this is a dispatch self.event_type = EventType.DISPATCH - logging.warning( - "event.json does not match pull_request or push:\n%s", - json.dumps(github_event, sort_keys=True, indent=4), - ) + print("PR Info:") + print(self) + else: + logging.warning( + "event.json does not match pull_request or push:\n%s", + json.dumps(github_event, sort_keys=True, indent=4), + ) self.sha = os.getenv( "GITHUB_SHA", "0000000000000000000000000000000000000000" ) From 952ab302ce1fa5d9b739bbdf6acdf2fcdd208a04 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 09:00:11 +0200 Subject: [PATCH 49/74] run test auto release by schedule --- .github/workflows/auto_release.yml | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index e0a94d3bbb1..7588e7998bf 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -7,8 +7,10 @@ env: concurrency: group: release on: # yamllint disable-line rule:truthy - # schedule: - # - cron: '0 10-16 * * 1-5' + # Workflow uses a test bucket for packages and dry run mode (no real releases) + schedule: + - cron: '0 9 * * *' + - cron: '0 15 * * *' workflow_dispatch: inputs: dry-run: @@ -19,6 +21,8 @@ on: # yamllint disable-line rule:truthy jobs: AutoRelease: + env: + DRY_RUN: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.dry_run != '' && github.event.inputs.dry_run || true }} runs-on: [self-hosted, release-maker] steps: - name: DebugInfo @@ -57,7 +61,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} @@ -65,7 +69,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} @@ -73,7 +77,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} @@ -81,7 +85,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} @@ -89,7 +93,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Post Slack Message if: ${{ !cancelled() }} From 5d09f205e5bb5ab8aa860b14334eb5656b2c2a1b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 19 Jul 2024 09:35:43 +0000 Subject: [PATCH 50/74] style fixes --- .github/workflows/auto_release.yml | 8 ++++++-- .github/workflows/create_release.yml | 5 +---- tests/ci/auto_release.py | 11 ++++++----- tests/ci/ci_buddy.py | 18 +++++++++--------- tests/ci/ci_utils.py | 11 +++++------ 5 files changed, 27 insertions(+), 26 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index 7588e7998bf..f2cbf771190 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -21,8 +21,6 @@ on: # yamllint disable-line rule:truthy jobs: AutoRelease: - env: - DRY_RUN: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.dry_run != '' && github.event.inputs.dry_run || true }} runs-on: [self-hosted, release-maker] steps: - name: DebugInfo @@ -34,6 +32,12 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK EOF + - name: Set DRY_RUN for schedule + if: ${{ github.event_name == 'schedule' }} + run: echo "DRY_RUN=true" >> "$GITHUB_ENV" + - name: Set DRY_RUN for dispatch + if: ${{ github.event_name == 'workflow_dispatch' }} + run: echo "DRY_RUN=${{ github.event.inputs.dry-run }}" >> "$GITHUB_ENV" - name: Check out repository code uses: ClickHouse/checkout@v1 with: diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 6246306e536..6d914d1567e 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -2,8 +2,7 @@ name: CreateRelease concurrency: group: release - -on: +'on': workflow_dispatch: inputs: ref: @@ -30,10 +29,8 @@ jobs: runs-on: [self-hosted, release-maker] steps: - name: DebugInfo - if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Check out repository code - if: ${{ ! inputs.autorelease }} uses: ClickHouse/checkout@v1 with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index b8f781c4d73..5d98d8810a4 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -55,7 +55,7 @@ class ReleaseParams: class AutoReleaseInfo: releases: List[ReleaseParams] - def add_release(self, release_params: ReleaseParams): + def add_release(self, release_params: ReleaseParams) -> None: self.releases.append(release_params) def dump(self): @@ -133,15 +133,16 @@ def _prepare(token): commit_sha = commit if commit_ci_status == SUCCESS: break - else: - print(f"CI status [{commit_ci_status}] - skip") + + print(f"CI status [{commit_ci_status}] - skip") commits_to_branch_head += 1 - ready = commit_ci_status == SUCCESS and commit_sha - if ready: + ready = False + if commit_ci_status == SUCCESS and commit_sha: print( f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" ) + ready = True else: print(f"WARNING: No ready commits found for release branch [{pr.head.ref}]") diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index 727a3d88359..ff8fdba2b6c 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -62,7 +62,7 @@ class CIBuddy: print(f"ERROR: Failed to post message, ex {e}") def _post_formatted( - self, title, body: Union[Dict, str], with_wf_link: bool + self, title: str, body: Union[Dict, str], with_wf_link: bool ) -> None: message = title if isinstance(body, dict): @@ -80,36 +80,36 @@ class CIBuddy: self.post(message) def post_info( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":white_circle: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_done( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":white_check_mark: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_warning( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":warning: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_critical( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":black_circle: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_job_error( self, - error_description, - job_name="", - with_instance_info=True, + error_description: str, + job_name: str = "", + with_instance_info: bool = True, with_wf_link: bool = True, - ): + ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: instance_id = Shell.run("ec2metadata --instance-id") or instance_id diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 4536d1f2b54..eb25a53d492 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -90,7 +90,7 @@ class GHActions: @staticmethod def get_commit_status_by_name( token: str, commit_sha: str, status_name: Union[str, Sequence] - ) -> Optional[str]: + ) -> str: assert len(token) == 40 assert len(commit_sha) == 40 assert is_hex(commit_sha) @@ -109,8 +109,8 @@ class GHActions: statuses = response.json() for status in statuses: if status["context"] in status_name: - return status["state"] - return None + return status["state"] # type: ignore + return "" @staticmethod def check_wf_completed(token: str, commit_sha: str) -> bool: @@ -135,10 +135,9 @@ class GHActions: f" Check workflow status: Check not completed [{check['name']}]" ) return False - else: - return True + return True except Exception as e: - print(f"ERROR: exception {e}") + print(f"ERROR: exception after attempt [{i}]: {e}") time.sleep(1) return False From 3b842885779e123af1fd8aeaca4e7c131c5a33a9 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 12:05:19 +0200 Subject: [PATCH 51/74] fix create release --- .github/workflows/create_release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 6d914d1567e..55644bdd503 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -39,6 +39,6 @@ jobs: uses: ./.github/actions/release with: ref: ${{ inputs.ref }} - type: inputs.type + type: ${{ inputs.type }} dry-run: ${{ inputs.dry-run }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} From 3767f723489507dad7fe275c245253e7885aab8e Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 20:43:14 +0200 Subject: [PATCH 52/74] more info for buddy --- .github/actions/release/action.yml | 20 +- .github/workflows/auto_release.yml | 3 +- tests/ci/artifactory.py | 86 ++++--- tests/ci/auto_release.py | 27 +++ tests/ci/ci_buddy.py | 4 +- tests/ci/ci_utils.py | 23 +- tests/ci/create_release.py | 348 ++++++++++++++++++----------- 7 files changed, 338 insertions(+), 173 deletions(-) diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml index fd4436f1f16..99ec02662f6 100644 --- a/.github/actions/release/action.yml +++ b/.github/actions/release/action.yml @@ -58,14 +58,11 @@ runs: shell: bash run: | python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Checkout master - shell: bash - run: | - git checkout master - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "update ChangeLog" [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv @@ -108,12 +105,13 @@ runs: shell: bash run: | git checkout "$GITHUB_REF_NAME" + # set current progress to OK + python3 ./tests/ci/create_release.py --set-progress-completed - name: Create GH Release shell: bash if: ${{ inputs.type == 'patch' }} run: | - python3 ./tests/ci/create_release.py --create-gh-release \ - ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash @@ -148,16 +146,26 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "docker server release" cd "./tests/ci" export CHECK_NAME="Docker server image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./tests/ci/create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "docker keeper release" cd "./tests/ci" export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./tests/ci/create_release.py --set-progress-completed + - name: Set Release progress completed + shell: bash + run: | + # If we here - set completed status, to post proper Slack OK or FAIL message in the next step + python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" + python3 ./tests/ci/create_release.py --set-progress-completed - name: Post Slack Message if: ${{ !cancelled() }} shell: bash diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index f2cbf771190..457ffacc7a8 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -102,7 +102,8 @@ jobs: - name: Post Slack Message if: ${{ !cancelled() }} run: | - echo Slack Message + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} - name: Clean up run: | docker ps --quiet | xargs --no-run-if-empty docker kill ||: diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 2009b122a18..98a0345c6bd 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -3,7 +3,12 @@ import time from pathlib import Path from typing import Optional from shutil import copy2 -from create_release import PackageDownloader, ReleaseInfo, ShellRunner +from create_release import ( + PackageDownloader, + ReleaseInfo, + ReleaseContextManager, + ReleaseProgress, +) from ci_utils import WithIter, Shell @@ -76,19 +81,20 @@ class R2MountPoint: ) _TEST_MOUNT_CMD = f"mount | grep -q {self.MOUNT_POINT}" - ShellRunner.run(_CLEAN_LOG_FILE_CMD) - ShellRunner.run(_UNMOUNT_CMD) - ShellRunner.run(_MKDIR_CMD) - ShellRunner.run(_MKDIR_FOR_CACHE) - ShellRunner.run(self.mount_cmd, async_=self.async_mount) + Shell.run(_CLEAN_LOG_FILE_CMD) + Shell.run(_UNMOUNT_CMD) + Shell.run(_MKDIR_CMD) + Shell.run(_MKDIR_FOR_CACHE) + # didn't manage to use simple run() and not block or fail + Shell.run_as_daemon(self.mount_cmd) if self.async_mount: time.sleep(3) - ShellRunner.run(_TEST_MOUNT_CMD) + Shell.run(_TEST_MOUNT_CMD, check=True) @classmethod def teardown(cls): print(f"Unmount [{cls.MOUNT_POINT}]") - ShellRunner.run(f"umount {cls.MOUNT_POINT}") + Shell.run(f"umount {cls.MOUNT_POINT}") class RepoCodenames(metaclass=WithIter): @@ -124,8 +130,8 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} includedeb {self.codename} {' '.join(paths)}" print("Running export command:") print(f" {cmd}") - ShellRunner.run(cmd) - ShellRunner.run("sync") + Shell.run(cmd, check=True) + Shell.run("sync") if self.codename == RepoCodenames.LTS: packages_with_version = [ @@ -137,8 +143,8 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} copy {RepoCodenames.STABLE} {RepoCodenames.LTS} {' '.join(packages_with_version)}" print("Running copy command:") print(f" {cmd}") - ShellRunner.run(cmd) - ShellRunner.run("sync") + Shell.run(cmd, check=True) + Shell.run("sync") def test_packages(self): Shell.run("docker pull ubuntu:latest") @@ -206,12 +212,12 @@ class RpmArtifactory: for command in commands: print("Running command:") print(f" {command}") - ShellRunner.run(command) + Shell.run(command, check=True) update_public_key = f"gpg --armor --export {self._SIGN_KEY}" pub_key_path = dest_dir / "repodata" / "repomd.xml.key" print("Updating repomd.xml.key") - pub_key_path.write_text(ShellRunner.run(update_public_key)[1]) + pub_key_path.write_text(Shell.run(update_public_key, check=True)) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) Shell.run("sync") @@ -264,23 +270,29 @@ class TgzArtifactory: if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - ShellRunner.run("sync") + Shell.run("sync") def test_packages(self): tgz_file = "/tmp/tmp.tgz" tgz_sha_file = "/tmp/tmp.tgz.sha512" - ShellRunner.run( - f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" + cmd = f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" + Shell.run( + cmd, + check=True, ) - ShellRunner.run( - f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512" + Shell.run( + f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512", + check=True, ) - expected_checksum = ShellRunner.run(f"cut -d ' ' -f 1 {tgz_sha_file}") - actual_checksum = ShellRunner.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") + expected_checksum = Shell.run(f"cut -d ' ' -f 1 {tgz_sha_file}", check=True) + actual_checksum = Shell.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") assert ( expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" - ShellRunner.run("rm /tmp/tmp.tgz*") + Shell.run("rm /tmp/tmp.tgz*") + release_info = ReleaseInfo.from_file() + release_info.tgz_command = cmd + release_info.dump() def parse_args() -> argparse.Namespace: @@ -338,20 +350,26 @@ if __name__ == "__main__": """ mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) if args.export_debian: - mp.init() - DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_DEB) as _: + mp.init() + DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.export_rpm: - mp.init() - RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_RPM) as _: + mp.init() + RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.export_tgz: - mp.init() - TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_TGZ) as _: + mp.init() + TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.test_debian: - DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_DEB) as _: + DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_tgz: - TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_TGZ) as _: + TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_rpm: - RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_RPM) as _: + RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 5d98d8810a4..39ab3156c80 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -25,11 +25,22 @@ def parse_args(): action="store_true", help="Post release branch statuses", ) + parser.add_argument( + "--post-auto-release-complete", + action="store_true", + help="Post autorelease completion status", + ) parser.add_argument( "--prepare", action="store_true", help="Prepare autorelease info", ) + parser.add_argument( + "--wf-status", + type=str, + default="", + help="overall workflow status [success|failure]", + ) return parser.parse_args(), parser @@ -180,6 +191,22 @@ def main(): title=f"Auto Release Status for {release_info.release_branch}", body=release_info.to_dict(), ) + if args.post_auto_release_complete: + assert args.wf_status, "--wf-status Required with --post-auto-release-complete" + if args.wf_status != SUCCESS: + CIBuddy(dry_run=False).post_job_error( + error_description="Autorelease workflow failed", + job_name="Autorelease", + with_instance_info=False, + with_wf_link=True, + critical=True, + ) + else: + CIBuddy(dry_run=False).post_info( + title=f"Autorelease completed", + body="", + with_wf_link=True, + ) elif args.prepare: _prepare(token=args.token or get_best_robot_token()) else: diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index ff8fdba2b6c..3eba5532e66 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -109,6 +109,7 @@ class CIBuddy: job_name: str = "", with_instance_info: bool = True, with_wf_link: bool = True, + critical: bool = False, ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: @@ -116,7 +117,8 @@ class CIBuddy: instance_type = Shell.run("ec2metadata --instance-type") or instance_type if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") - line_err = f":red_circle: *Error: {error_description}*\n\n" + sign = ":red_circle:" if not critical else ":black_circle:" + line_err = f"{sign} *Error: {error_description}*\n\n" line_ghr = f" *Runner:* `{instance_type}`, `{instance_id}`\n" line_job = f" *Job:* `{job_name}`\n" line_pr_ = f" *PR:* , <{self.commit_url}|{self.sha}>\n" diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index eb25a53d492..efbf014cd52 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -4,7 +4,7 @@ import subprocess import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Sequence +from typing import Any, Iterator, List, Union, Optional, Sequence, Tuple import requests @@ -142,6 +142,16 @@ class GHActions: return False + @staticmethod + def get_pr_url_by_branch(repo, branch): + get_url_cmd = ( + f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" + ) + url = Shell.run(get_url_cmd) + if not url: + print(f"ERROR: PR nor found, branch [{branch}]") + return url + class Shell: @classmethod @@ -157,7 +167,10 @@ class Shell: return res.stdout.strip() @classmethod - def run(cls, command, check=False): + def run(cls, command, check=False, dry_run=False): + if dry_run: + print(f"Dry-ryn. Would run command [{command}]") + return "" print(f"Run command [{command}]") res = "" result = subprocess.run( @@ -178,6 +191,12 @@ class Shell: assert result.returncode == 0 return res.strip() + @classmethod + def run_as_daemon(cls, command): + print(f"Run daemon command [{command}]") + subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with + return 0, "" + @classmethod def check(cls, command): result = subprocess.run( diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 414ec8afd3e..4347cfebb54 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -2,7 +2,6 @@ import argparse import dataclasses import json import os -import subprocess from contextlib import contextmanager from copy import copy @@ -13,7 +12,7 @@ from git_helper import Git, GIT_PREFIX from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper -from ci_utils import Shell +from ci_utils import Shell, GHActions from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, @@ -31,32 +30,62 @@ CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) RELEASE_INFO_FILE = "/tmp/release_info.json" -class ShellRunner: +class ReleaseProgress: + STARTED = "started" + DOWNLOAD_PACKAGES = "download packages" + PUSH_RELEASE_TAG = "push release tag" + PUSH_NEW_RELEASE_BRANCH = "push new release branch" + BUMP_VERSION = "bump version" + CREATE_GH_RELEASE = "create GH release" + EXPORT_TGZ = "export TGZ packages" + EXPORT_RPM = "export RPM packages" + EXPORT_DEB = "export DEB packages" + TEST_TGZ = "test TGZ packages" + TEST_RPM = "test RPM packages" + TEST_DEB = "test DEB packages" - @classmethod - def run( - cls, command, check_retcode=True, print_output=True, async_=False, dry_run=False - ): - if dry_run: - print(f"Dry-run: Would run shell command: [{command}]") - return 0, "" - print(f"Running shell command: [{command}]") - if async_: - subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with - return 0, "" - result = subprocess.run( - command + " 2>&1", - shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=True, - ) - if print_output: - print(result.stdout) - if check_retcode: - assert result.returncode == 0, f"Return code [{result.returncode}]" - return result.returncode, result.stdout + +class ReleaseProgressDescription: + OK = "OK" + FAILED = "FAILED" + + +class ReleaseContextManager: + def __init__(self, release_progress): + self.release_progress = release_progress + self.release_info = None + + def __enter__(self): + if self.release_progress == ReleaseProgress.STARTED: + # create initial release info + self.release_info = ReleaseInfo( + release_branch="NA", + commit_sha=args.ref, + release_tag="NA", + version="NA", + codename="NA", + previous_release_tag="NA", + previous_release_sha="NA", + release_progress=ReleaseProgress.STARTED, + ).dump() + else: + # fetch release info from fs and update + self.release_info = ReleaseInfo.from_file() + assert self.release_info + assert ( + self.release_info.progress_description == ReleaseProgressDescription.OK + ), "Must be OK on the start of new context" + self.release_info.release_progress = self.release_progress + self.release_info.dump() + return self.release_info + + def __exit__(self, exc_type, exc_value, traceback): + assert self.release_info + if exc_type is not None: + self.release_info.progress_description = ReleaseProgressDescription.FAILED + else: + self.release_info.progress_description = ReleaseProgressDescription.OK + self.release_info.dump() @dataclasses.dataclass @@ -74,6 +103,10 @@ class ReleaseInfo: release_url: str = "" debian_command: str = "" rpm_command: str = "" + tgz_command: str = "" + docker_command: str = "" + release_progress: str = "" + progress_description: str = "" @staticmethod def from_file() -> "ReleaseInfo": @@ -85,9 +118,9 @@ class ReleaseInfo: print(f"Dump release info into [{RELEASE_INFO_FILE}]") with open(RELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + return self - @staticmethod - def prepare(commit_ref: str, release_type: str) -> None: + def prepare(self, commit_ref: str, release_type: str) -> "ReleaseInfo": version = None release_branch = None release_tag = None @@ -97,8 +130,9 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - ShellRunner.run( - f"git merge-base --is-ancestor origin/{commit_ref} origin/master" + Shell.run( + f"git merge-base --is-ancestor origin/{commit_ref} origin/master", + check=True, ) with checkout(commit_ref): commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) @@ -130,10 +164,11 @@ class ReleaseInfo: version.with_description(codename) release_branch = f"{version.major}.{version.minor}" release_tag = version.describe - ShellRunner.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags") + Shell.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags", check=True) # check commit is right and on a right branch - ShellRunner.run( - f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}" + Shell.run( + f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}", + check=True, ) if version.patch == 1: expected_version = copy(version) @@ -172,21 +207,22 @@ class ReleaseInfo: and version and codename in ("lts", "stable") ) - res = ReleaseInfo( - release_branch=release_branch, - commit_sha=commit_sha, - release_tag=release_tag, - version=version.string, - codename=codename, - previous_release_tag=previous_release_tag, - previous_release_sha=previous_release_sha, - ) - res.dump() + + self.release_branch = release_branch + self.commit_sha = commit_sha + self.release_tag = release_tag + self.version = version.string + self.codename = codename + self.previous_release_tag = previous_release_tag + self.previous_release_sha = previous_release_sha + self.release_progress = ReleaseProgress.STARTED + self.progress_description = ReleaseProgressDescription.OK + return self def push_release_tag(self, dry_run: bool) -> None: if dry_run: # remove locally created tag from prev run - ShellRunner.run( + Shell.run( f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag} ||:" ) # Create release tag @@ -194,16 +230,17 @@ class ReleaseInfo: f"Create and push release tag [{self.release_tag}], commit [{self.commit_sha}]" ) tag_message = f"Release {self.release_tag}" - ShellRunner.run( - f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}" + Shell.run( + f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}", + check=True, ) cmd_push_tag = f"{GIT_PREFIX} push origin {self.release_tag}:{self.release_tag}" - ShellRunner.run(cmd_push_tag, dry_run=dry_run) + Shell.run(cmd_push_tag, dry_run=dry_run, check=True) @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - ShellRunner.run(cmd, dry_run=dry_run) + Shell.run(cmd, dry_run=dry_run, check=True) def push_new_release_branch(self, dry_run: bool) -> None: assert ( @@ -220,8 +257,8 @@ class ReleaseInfo: ), f"Unexpected current version in git, must precede [{self.version}] by one step, actual [{version.string}]" if dry_run: # remove locally created branch from prev run - ShellRunner.run( - f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch} ||:" + Shell.run( + f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch}" ) print( f"Create and push new release branch [{new_release_branch}], commit [{self.commit_sha}]" @@ -234,7 +271,7 @@ class ReleaseInfo: cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) - ShellRunner.run(cmd_push_branch, dry_run=dry_run) + Shell.run(cmd_push_branch, dry_run=dry_run, check=True) print("Create and push backport tags for new release branch") ReleaseInfo._create_gh_label( @@ -243,12 +280,13 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - ShellRunner.run( + Shell.run( f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, dry_run=dry_run, + check=True, ) def update_version_and_contributors_list(self, dry_run: bool) -> None: @@ -274,31 +312,34 @@ class ReleaseInfo: body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file '{body_file} --label 'do not test' --assignee @{actor}" - ShellRunner.run(cmd_commit_version_upd, dry_run=dry_run) - ShellRunner.run(cmd_push_branch, dry_run=dry_run) - ShellRunner.run(cmd_create_pr, dry_run=dry_run) + Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) + Shell.run(cmd_push_branch, check=True, dry_run=dry_run) + Shell.run(cmd_create_pr, check=True, dry_run=dry_run) if dry_run: - ShellRunner.run( - f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" - ) - ShellRunner.run( + Shell.run(f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'") + Shell.run( f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" ) + self.version_bump_pr = GHActions.get_pr_url_by_branch( + repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors + ) - def update_release_info(self, dry_run: bool) -> None: + def update_release_info(self, dry_run: bool) -> "ReleaseInfo": branch = f"auto/{release_info.release_tag}" if not dry_run: - get_url_cmd = f"gh pr list --repo {GITHUB_REPOSITORY} --head {branch} --json url --jq '.[0].url'" - url = Shell.run(get_url_cmd) - if url: - print(f"Update release info with Changelog PR link [{url}]") - else: - print(f"WARNING: Changelog PR not found, branch [{branch}]") + url = GHActions.get_pr_url_by_branch(repo=GITHUB_REPOSITORY, branch=branch) else: url = "dry-run" + print(f"ChangeLog PR url [{url}]") self.changelog_pr = url + print(f"Release url [{url}]") + self.release_url = ( + f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + ) + self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.release_branch} clickhouse --version" self.dump() + return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: repo = os.getenv("GITHUB_REPOSITORY") @@ -376,7 +417,7 @@ class PackageDownloader: self.macos_package_files = ["clickhouse-macos", "clickhouse-macos-aarch64"] self.file_to_type = {} - ShellRunner.run(f"mkdir -p {self.LOCAL_DIR}") + Shell.run(f"mkdir -p {self.LOCAL_DIR}") for package_type in self.PACKAGE_TYPES: for package in self.package_names: @@ -426,7 +467,7 @@ class PackageDownloader: return res def run(self): - ShellRunner.run(f"rm -rf {self.LOCAL_DIR}/*") + Shell.run(f"rm -rf {self.LOCAL_DIR}/*") for package_file in ( self.deb_package_files + self.rpm_package_files + self.tgz_package_files ): @@ -499,6 +540,37 @@ class PackageDownloader: return True +@contextmanager +def checkout(ref: str) -> Iterator[None]: + orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" + assert orig_ref + if ref not in (orig_ref,): + Shell.run(f"{GIT_PREFIX} checkout {ref}") + try: + yield + except (Exception, KeyboardInterrupt) as e: + print(f"ERROR: Exception [{e}]") + Shell.run(rollback_cmd) + raise + Shell.run(rollback_cmd) + + +@contextmanager +def checkout_new(ref: str) -> Iterator[None]: + orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" + assert orig_ref + Shell.run(f"{GIT_PREFIX} checkout -b {ref}", check=True) + try: + yield + except (Exception, KeyboardInterrupt) as e: + print(f"ERROR: Exception [{e}]") + Shell.run(rollback_cmd) + raise + Shell.run(rollback_cmd) + + def parse_args() -> argparse.Namespace: parser = argparse.ArgumentParser( formatter_class=argparse.ArgumentDefaultsHelpFormatter, @@ -556,44 +628,26 @@ def parse_args() -> argparse.Namespace: action="store_true", help="do not make any actual changes in the repo, just show what will be done", ) - + parser.add_argument( + "--set-progress-started", + action="store_true", + help="Set new progress step, --progress must be set", + ) + parser.add_argument( + "--progress", + type=str, + help="Progress step name, see @ReleaseProgress", + ) + parser.add_argument( + "--set-progress-completed", + action="store_true", + help="Set current progress step to OK (completed)", + ) return parser.parse_args() -@contextmanager -def checkout(ref: str) -> Iterator[None]: - _, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD") - rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" - assert orig_ref - if ref not in (orig_ref,): - Shell.run(f"{GIT_PREFIX} checkout {ref}") - try: - yield - except (Exception, KeyboardInterrupt) as e: - print(f"ERROR: Exception [{e}]") - ShellRunner.run(rollback_cmd) - raise - ShellRunner.run(rollback_cmd) - - -@contextmanager -def checkout_new(ref: str) -> Iterator[None]: - _, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD") - rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" - assert orig_ref - ShellRunner.run(f"{GIT_PREFIX} checkout -b {ref}") - try: - yield - except (Exception, KeyboardInterrupt) as e: - print(f"ERROR: Exception [{e}]") - ShellRunner.run(rollback_cmd) - raise - ShellRunner.run(rollback_cmd) - - if __name__ == "__main__": args = parse_args() - assert args.dry_run # prepare ssh for git if needed _ssh_agent = None @@ -605,35 +659,56 @@ if __name__ == "__main__": _ssh_agent.print_keys() if args.prepare_release_info: - assert ( - args.ref and args.release_type - ), "--ref and --release-type must be provided with --prepare-release-info" - ReleaseInfo.prepare(commit_ref=args.ref, release_type=args.release_type) - if args.push_release_tag: - release_info = ReleaseInfo.from_file() - release_info.push_release_tag(dry_run=args.dry_run) - if args.push_new_release_branch: - release_info = ReleaseInfo.from_file() - release_info.push_new_release_branch(dry_run=args.dry_run) - if args.create_bump_version_pr: - # TODO: store link to PR in release info - release_info = ReleaseInfo.from_file() - release_info.update_version_and_contributors_list(dry_run=args.dry_run) + with ReleaseContextManager( + release_progress=ReleaseProgress.STARTED + ) as release_info: + assert ( + args.ref and args.release_type + ), "--ref and --release-type must be provided with --prepare-release-info" + release_info.prepare(commit_ref=args.ref, release_type=args.release_type) + if args.download_packages: - release_info = ReleaseInfo.from_file() - p = PackageDownloader( - release=release_info.release_branch, - commit_sha=release_info.commit_sha, - version=release_info.version, - ) - p.run() + with ReleaseContextManager( + release_progress=ReleaseProgress.DOWNLOAD_PACKAGES + ) as release_info: + p = PackageDownloader( + release=release_info.release_branch, + commit_sha=release_info.commit_sha, + version=release_info.version, + ) + p.run() + + if args.push_release_tag: + with ReleaseContextManager( + release_progress=ReleaseProgress.PUSH_RELEASE_TAG + ) as release_info: + release_info.push_release_tag(dry_run=args.dry_run) + + if args.push_new_release_branch: + with ReleaseContextManager( + release_progress=ReleaseProgress.PUSH_NEW_RELEASE_BRANCH + ) as release_info: + release_info.push_new_release_branch(dry_run=args.dry_run) + + if args.create_bump_version_pr: + with ReleaseContextManager( + release_progress=ReleaseProgress.BUMP_VERSION + ) as release_info: + release_info.update_version_and_contributors_list(dry_run=args.dry_run) + if args.create_gh_release: - release_info = ReleaseInfo.from_file() - p = PackageDownloader( - release=release_info.release_branch, - commit_sha=release_info.commit_sha, - version=release_info.version, - ) + with ReleaseContextManager( + release_progress=ReleaseProgress.CREATE_GH_RELEASE + ) as release_info: + p = PackageDownloader( + release=release_info.release_branch, + commit_sha=release_info.commit_sha, + version=release_info.version, + ) + release_info.create_gh_release( + packages_files=p.get_all_packages_files(), dry_run=args.dry_run + ) + if args.post_status: release_info = ReleaseInfo.from_file() release_info.update_release_info(dry_run=args.dry_run) @@ -646,6 +721,21 @@ if __name__ == "__main__": f"Failed to issue new release", dataclasses.asdict(release_info) ) + if args.set_progress_started: + ri = ReleaseInfo.from_file() + ri.release_progress = args.progress + ri.progress_description = ReleaseProgressDescription.FAILED + ri.dump() + assert args.progress, "Progress step name must be provided" + + if args.set_progress_completed: + ri = ReleaseInfo.from_file() + assert ( + ri.progress_description == ReleaseProgressDescription.FAILED + ), "Must be FAILED before set to OK" + ri.progress_description = ReleaseProgressDescription.OK + ri.dump() + # tear down ssh if _ssh_agent and _key_pub: _ssh_agent.remove(_key_pub) From ee3c0e7e1f37a4e9388866145e9cbb8f0220e42c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 21:50:53 +0200 Subject: [PATCH 53/74] Better diagnostics in `test_disk_configuration` --- .../test_disk_configuration/test.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index c003ff85755..afc5303298c 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -208,13 +208,21 @@ def test_merge_tree_custom_disk_setting(start_cluster): secret_access_key='minio123'); """ ) - count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + + list1 = list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + count1 = len(list1) + node1.query(f"INSERT INTO {TABLE_NAME}_3 SELECT number FROM numbers(100)") assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_3")) == 100 - assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) - == count - ) + + list2 = list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + count2 = len(list2) + + if count1 != count2: + print("list1: ", list1) + print("list2: ", list2) + + assert count1 == count2 assert ( len(list(minio.list_objects(cluster.minio_bucket, "data2/", recursive=True))) > 0 From f9b97aac84d6ab5f377261e5b57c2d6e8a432cef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 23:19:33 +0200 Subject: [PATCH 54/74] Fix bad test `02950_part_log_bytes_uncompressed` --- .../0_stateless/02950_part_log_bytes_uncompressed.sql | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql index 248475ab84b..cfed02eaeeb 100644 --- a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql +++ b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql @@ -1,3 +1,6 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings +-- Because we compare part sizes, and they could be affected by index granularity and index compression settings. + CREATE TABLE part_log_bytes_uncompressed ( key UInt8, value UInt8 @@ -17,7 +20,8 @@ ALTER TABLE part_log_bytes_uncompressed DROP PART 'all_4_4_0' SETTINGS mutations SYSTEM FLUSH LOGS; -SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed FROM system.part_log +SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed)) +FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'part_log_bytes_uncompressed' AND (event_type != 'RemovePart' OR part_name = 'all_4_4_0') -- ignore removal of other parts ORDER BY part_name, event_type; From 67f51153661f37370f4e365ed3f19c6a4b84c2f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 04:16:31 +0200 Subject: [PATCH 55/74] Update 02950_part_log_bytes_uncompressed.sql --- tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql index cfed02eaeeb..24425062116 100644 --- a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql +++ b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql @@ -20,7 +20,7 @@ ALTER TABLE part_log_bytes_uncompressed DROP PART 'all_4_4_0' SETTINGS mutations SYSTEM FLUSH LOGS; -SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed)) +SELECT event_type, table, part_name, bytes_uncompressed > 0, (bytes_uncompressed > 0 ? (size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed))) : '0') FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'part_log_bytes_uncompressed' AND (event_type != 'RemovePart' OR part_name = 'all_4_4_0') -- ignore removal of other parts From f1021b70f7f79fc6f921989573435ab5df406bdc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 07:05:09 +0200 Subject: [PATCH 56/74] Better diagnostics for test trace_events_stress --- src/Common/TraceSender.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 91d07367a82..064da1b3d76 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -23,8 +23,15 @@ namespace DB LazyPipeFDs TraceSender::pipe; +static thread_local bool inside_send = false; void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras) { + DENY_ALLOCATIONS_IN_SCOPE; + + if (unlikely(inside_send)) + abort(); /// The method shouldn't be called recursively or throw exceptions. + inside_send = true; + constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + sizeof(UInt8) /// String size + QUERY_ID_MAX_LEN /// Maximum query_id length @@ -80,6 +87,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Ext writePODBinary(extras.increment, out); out.next(); + + inside_send = false; } } From 6437088d81200909245172c441de3bf301f553b0 Mon Sep 17 00:00:00 2001 From: Mikhail Date: Sun, 21 Jul 2024 12:52:26 +0700 Subject: [PATCH 57/74] translate playground page to russian in ru locale --- docs/ru/getting-started/playground.md | 40 +++++++++++++-------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/ru/getting-started/playground.md b/docs/ru/getting-started/playground.md index c8c987eec9e..eb990c6071e 100644 --- a/docs/ru/getting-started/playground.md +++ b/docs/ru/getting-started/playground.md @@ -1,43 +1,43 @@ --- slug: /ru/getting-started/playground sidebar_position: 14 -sidebar_label: Playground +sidebar_label: Песочница --- -# ClickHouse Playground {#clickhouse-playground} +# Песочница ClickHouse {#clickhouse-playground} -[ClickHouse Playground](https://play.clickhouse.com/play?user=play) allows people to experiment with ClickHouse by running queries instantly, without setting up their server or cluster. -Several example datasets are available in Playground. +[Песочница ClickHouse](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. +В Песочнице доступны несколько примеров наборов данных. -You can make queries to Playground using any HTTP client, for example [curl](https://curl.haxx.se) or [wget](https://www.gnu.org/software/wget/), or set up a connection using [JDBC](../interfaces/jdbc.md) or [ODBC](../interfaces/odbc.md) drivers. More information about software products that support ClickHouse is available [here](../interfaces/index.md). +Вы можете выполнять запросы к Песочнице, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). -## Credentials {#credentials} +## Учетные данные {#credentials} -| Parameter | Value | +| Параметр | Значение | |:--------------------|:-----------------------------------| -| HTTPS endpoint | `https://play.clickhouse.com:443/` | -| Native TCP endpoint | `play.clickhouse.com:9440` | -| User | `explorer` or `play` | -| Password | (empty) | +| HTTPS-адрес | `https://play.clickhouse.com:443/` | +| TCP-адрес | `play.clickhouse.com:9440` | +| Пользователь | `explorer` или `play` | +| Пароль | (пусто) | -## Limitations {#limitations} +## Ограничения {#limitations} -The queries are executed as a read-only user. It implies some limitations: +Запросы выполняются от имени пользователя с правами только на чтение. Это предполагает некоторые ограничения: -- DDL queries are not allowed -- INSERT queries are not allowed +- DDL-запросы не разрешены +- INSERT-запросы не разрешены -The service also have quotas on its usage. +Сервис также имеет квоты на использование. -## Examples {#examples} +## Примеры {#examples} -HTTPS endpoint example with `curl`: +Пример использования HTTPS-адреса с `curl`: -``` bash +```bash curl "https://play.clickhouse.com/?user=explorer" --data-binary "SELECT 'Play ClickHouse'" ``` -TCP endpoint example with [CLI](../interfaces/cli.md): +Пример использования TCP-адреса с [CLI](../interfaces/cli.md): ``` bash clickhouse client --secure --host play.clickhouse.com --user explorer From e74892bfaf4aa98654a6253a34bbeb3f1740de41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 08:21:09 +0200 Subject: [PATCH 58/74] Update playground.md --- docs/ru/getting-started/playground.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/getting-started/playground.md b/docs/ru/getting-started/playground.md index eb990c6071e..a2d5498fb9a 100644 --- a/docs/ru/getting-started/playground.md +++ b/docs/ru/getting-started/playground.md @@ -1,15 +1,15 @@ --- slug: /ru/getting-started/playground sidebar_position: 14 -sidebar_label: Песочница +sidebar_label: Playground --- -# Песочница ClickHouse {#clickhouse-playground} +# ClickHouse Playground {#clickhouse-playground} -[Песочница ClickHouse](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. -В Песочнице доступны несколько примеров наборов данных. +[ClickHouse Playground](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. +В Playground доступны несколько примеров наборов данных. -Вы можете выполнять запросы к Песочнице, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). +Вы можете выполнять запросы к Playground, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). ## Учетные данные {#credentials} From a371557f07ab97b88da40e6b790a5d361f437a47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 09:06:45 +0200 Subject: [PATCH 59/74] Make test `00997_set_index_array` lighter --- tests/queries/0_stateless/00997_set_index_array.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00997_set_index_array.sql b/tests/queries/0_stateless/00997_set_index_array.sql index d6d27f5a6a0..ed972d1a545 100644 --- a/tests/queries/0_stateless/00997_set_index_array.sql +++ b/tests/queries/0_stateless/00997_set_index_array.sql @@ -12,10 +12,10 @@ ORDER BY (primary_key); INSERT INTO set_array select - toString(intDiv(number, 1000000)) as primary_key, + toString(intDiv(number, 100000)) as primary_key, array(number) as index_array from system.numbers -limit 10000000; +limit 1000000; OPTIMIZE TABLE set_array FINAL; From ea83f89374f723d1d53dca64ec58e8fd993f013d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 10:39:03 +0200 Subject: [PATCH 60/74] Fix inconsistent formatting of lambda functions inside composite types --- src/Parsers/ASTFunction.cpp | 8 ++++---- ...ite_expressions_lambda_consistent_formatting.reference | 0 ...composite_expressions_lambda_consistent_formatting.sql | 6 ++++++ 3 files changed, 10 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference create mode 100644 tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f39229d7566..b04ec1c22b2 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -19,9 +18,6 @@ #include #include #include -#include - -#include using namespace std::literals; @@ -632,6 +628,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : ""); @@ -642,12 +639,14 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format { settings.ostr << (settings.hilite ? hilite_operator : "") << ((frame.need_parens && !alias.empty()) ? "tuple" : "") << '(' << (settings.hilite ? hilite_none : ""); + for (size_t i = 0; i < arguments->children.size(); ++i) { if (i != 0) settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : ""); @@ -663,6 +662,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : ""); diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql new file mode 100644 index 00000000000..42c823cf476 --- /dev/null +++ b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql @@ -0,0 +1,6 @@ +SELECT [1, (x -> 1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT (1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT map(1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT [1, lambda(x, 1)]; -- { serverError UNKNOWN_IDENTIFIER } +SELECT (1, lambda(x, 1)); -- { serverError UNKNOWN_IDENTIFIER } +SELECT map(1, lambda(x, 1)); -- { serverError UNKNOWN_IDENTIFIER } From 433ac55d13276e7a0b26ded07f27f17726703dab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 21 Jul 2024 11:41:42 +0200 Subject: [PATCH 61/74] Correctly handle failure --- src/Common/Allocator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 7f2241ab4c0..1b43e746c69 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -206,7 +206,7 @@ void * Allocator::realloc(void * buf, size_t old_size, } else { - [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size); ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); } } @@ -239,7 +239,7 @@ void * Allocator::realloc(void * buf, size_t old_size, void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) { - [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size); throw DB::ErrnoException( DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot realloc from {} to {}", From 08353419d0798f209627f16f840f4d101227431d Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 11:50:15 +0200 Subject: [PATCH 62/74] CI: Never await on BuildReport job, Skip BuildReport if no builds in workflow --- tests/ci/ci_cache.py | 27 ++++++++++---- tests/ci/ci_definitions.py | 3 ++ tests/ci/merge_pr.py | 5 ++- tests/ci/test_ci_config.py | 74 ++++++++++++++++++++++++++++++++++---- 4 files changed, 96 insertions(+), 13 deletions(-) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index cfefb954fcd..16b6eac1ecb 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -638,7 +638,7 @@ class CiCache: pushes pending records for all jobs that supposed to be run """ for job, job_config in self.jobs_to_do.items(): - if not job_config.has_digest(): + if not job_config.has_digest() or job_config.disable_await: continue pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL) assert job_config.batches @@ -708,7 +708,7 @@ class CiCache: Filter is to be applied in PRs to remove jobs that are not affected by the change :return: """ - remove_from_to_do = [] + remove_from_workflow = [] required_builds = [] has_test_jobs_to_skip = False for job_name, job_config in self.jobs_to_do.items(): @@ -723,26 +723,41 @@ class CiCache: job=reference_name, job_config=reference_config, ): - remove_from_to_do.append(job_name) + remove_from_workflow.append(job_name) has_test_jobs_to_skip = True else: required_builds += ( job_config.required_builds if job_config.required_builds else [] ) if has_test_jobs_to_skip: - # If there are tests to skip, it means build digest has not been changed. + # If there are tests to skip, it means builds are not affected as well. # No need to test builds. Let's keep all builds required for test jobs and skip the others for job_name, job_config in self.jobs_to_do.items(): if CI.is_build_job(job_name): if job_name not in required_builds: - remove_from_to_do.append(job_name) + remove_from_workflow.append(job_name) - for job in remove_from_to_do: + for job in remove_from_workflow: print(f"Filter job [{job}] - not affected by the change") if job in self.jobs_to_do: del self.jobs_to_do[job] if job in self.jobs_to_wait: del self.jobs_to_wait[job] + if job in self.jobs_to_skip: + self.jobs_to_skip.remove(job) + + # special handling for the special job: BUILD_CHECK + has_builds = False + for job in list(self.jobs_to_do) + self.jobs_to_skip: + if CI.is_build_job(job): + has_builds = True + break + if not has_builds: + if CI.JobNames.BUILD_CHECK in self.jobs_to_do: + print( + f"Filter job [{CI.JobNames.BUILD_CHECK}] - no builds are required in the workflow" + ) + del self.jobs_to_do[CI.JobNames.BUILD_CHECK] def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None: """ diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 4c78efd39a2..a8d9793f1d3 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -351,6 +351,8 @@ class JobConfig: run_by_label: str = "" # to run always regardless of the job digest or/and label run_always: bool = False + # disables CI await for a given job + disable_await: bool = False # if the job needs to be run on the release branch, including master (building packages, docker server). # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. required_on_release_branch: bool = False @@ -395,6 +397,7 @@ class CommonJobConfigs: ], ), runner_type=Runners.STYLE_CHECKER_ARM, + disable_await=True, ) COMPATIBILITY_TEST = JobConfig( job_name_keyword="compatibility", diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 4d8facafb84..13c7537a84b 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -254,11 +254,14 @@ def main(): statuses = get_commit_filtered_statuses(commit) has_failed_statuses = False + has_native_failed_status = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") if CI.is_required(status.context) and status.state != SUCCESS: print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True + if status.context != CI.StatusNames.SYNC: + has_native_failed_status = True if args.wf_status == SUCCESS or has_failed_statuses: # set Mergeable check if workflow is successful (green) @@ -280,7 +283,7 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - if args.wf_status == SUCCESS and not has_failed_statuses: + if args.wf_status == SUCCESS and not has_native_failed_status: sys.exit(0) else: sys.exit(1) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 12e863c4d8d..10867ea1444 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -587,11 +587,11 @@ class TestCIConfig(unittest.TestCase): for job, job_config in ci_cache.jobs_to_do.items(): if job in MOCK_AFFECTED_JOBS: MOCK_REQUIRED_BUILDS += job_config.required_builds - elif job not in MOCK_AFFECTED_JOBS: + elif job not in MOCK_AFFECTED_JOBS and not job_config.disable_await: ci_cache.jobs_to_wait[job] = job_config for job, job_config in ci_cache.jobs_to_do.items(): - if job_config.reference_job_name: + if job_config.reference_job_name or job_config.disable_await: # jobs with reference_job_name in config are not supposed to have records in the cache - continue continue if job in MOCK_AFFECTED_JOBS: @@ -624,11 +624,73 @@ class TestCIConfig(unittest.TestCase): + MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS ) + self.assertTrue(CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, "We must never await on Builds Report") self.assertCountEqual( list(ci_cache.jobs_to_wait), - [ - CI.JobNames.BUILD_CHECK, - ] - + MOCK_REQUIRED_BUILDS, + MOCK_REQUIRED_BUILDS, + ) + self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) + + def test_ci_py_filters_not_affected_jobs_in_prs_no_builds(self): + """ + checks ci.py filters not affected jobs in PRs, no builds required + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 123 + assert pr_info.is_pr + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + assert not ci_cache.jobs_to_wait + assert not ci_cache.jobs_to_skip + + MOCK_AFFECTED_JOBS = [ + CI.JobNames.DOCS_CHECK, + ] + MOCK_REQUIRED_BUILDS = [] + + # pretend there are pending jobs that we need to wait + for job, job_config in ci_cache.jobs_to_do.items(): + if job in MOCK_AFFECTED_JOBS: + if job_config.required_builds: + MOCK_REQUIRED_BUILDS += job_config.required_builds + elif job not in MOCK_AFFECTED_JOBS and not job_config.disable_await: + ci_cache.jobs_to_wait[job] = job_config + + for job, job_config in ci_cache.jobs_to_do.items(): + if job_config.reference_job_name or job_config.disable_await: + # jobs with reference_job_name in config are not supposed to have records in the cache - continue + continue + if job in MOCK_AFFECTED_JOBS: + continue + for batch in range(job_config.num_batches): + # add any record into cache + record = CiCache.Record( + record_type=random.choice( + [ + CiCache.RecordType.FAILED, + CiCache.RecordType.PENDING, + CiCache.RecordType.SUCCESSFUL, + ] + ), + job_name=job, + job_digest=ci_cache.job_digests[job], + batch=batch, + num_batches=job_config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == CiCache.RecordType.FAILED.value: + records_[record.to_str_key()] = record + + ci_cache.filter_out_not_affected_jobs() + expected_to_do = MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS + self.assertCountEqual( + list(ci_cache.jobs_to_wait), + MOCK_REQUIRED_BUILDS, ) self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) From c61581ca69757ce9c1d15b6ccc139d48d27f5b07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 12:06:53 +0200 Subject: [PATCH 63/74] Fix error --- src/Common/TraceSender.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 064da1b3d76..f1adf7c516a 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -26,11 +26,16 @@ LazyPipeFDs TraceSender::pipe; static thread_local bool inside_send = false; void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras) { - DENY_ALLOCATIONS_IN_SCOPE; - + /** The method shouldn't be called recursively or throw exceptions. + * There are several reasons: + * - avoid infinite recursion when some of subsequent functions invoke tracing; + * - avoid inconsistent writes if the method was interrupted by a signal handler in the middle of writing, + * and then another tracing is invoked (e.g., from query profiler). + */ if (unlikely(inside_send)) - abort(); /// The method shouldn't be called recursively or throw exceptions. + return; inside_send = true; + DENY_ALLOCATIONS_IN_SCOPE; constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + sizeof(UInt8) /// String size From 02bf9e4243c329611fc0be43432d3f9c290512d5 Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 12:46:58 +0200 Subject: [PATCH 64/74] push pending records before await call --- tests/ci/ci.py | 5 +- tests/ci/ci_cache.py | 93 +++++++++++++++++++++++++++++++++++++- tests/ci/test_ci_config.py | 5 +- 3 files changed, 99 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index ff31d1ce489..f6bec5304a0 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1110,13 +1110,14 @@ def main() -> int: ci_cache.print_status() if IS_CI and not pr_info.is_merge_queue: - # wait for pending jobs to be finished, await_jobs is a long blocking call - ci_cache.await_pending_jobs(pr_info.is_release) if pr_info.is_release: print("Release/master: CI Cache add pending records for all todo jobs") ci_cache.push_pending_all(pr_info.is_release) + # wait for pending jobs to be finished, await_jobs is a long blocking call + ci_cache.await_pending_jobs(pr_info.is_release) + # conclude results result["git_ref"] = git_ref result["version"] = version diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 16b6eac1ecb..85eabb84f9f 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -638,7 +638,14 @@ class CiCache: pushes pending records for all jobs that supposed to be run """ for job, job_config in self.jobs_to_do.items(): - if not job_config.has_digest() or job_config.disable_await: + if ( + job in self.jobs_to_wait + or not job_config.has_digest() + or job_config.disable_await + ): + # 1. "job in self.jobs_to_wait" - this job already has a pending record in cache + # 2. "not job_config.has_digest()" - cache is not used for these jobs + # 3. "job_config.disable_await" - await is explicitly disabled continue pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL) assert job_config.batches @@ -899,3 +906,87 @@ class CiCache: self.jobs_to_wait[job] = job_config return self + + +if __name__ == "__main__": + # for testing + job_digest = { + "package_release": "bbbd3519d1", + "package_aarch64": "bbbd3519d1", + "package_asan": "bbbd3519d1", + "package_ubsan": "bbbd3519d1", + "package_tsan": "bbbd3519d1", + "package_msan": "bbbd3519d1", + "package_debug": "bbbd3519d1", + "package_release_coverage": "bbbd3519d1", + "binary_release": "bbbd3519d1", + "binary_tidy": "bbbd3519d1", + "binary_darwin": "bbbd3519d1", + "binary_aarch64": "bbbd3519d1", + "binary_aarch64_v80compat": "bbbd3519d1", + "binary_freebsd": "bbbd3519d1", + "binary_darwin_aarch64": "bbbd3519d1", + "binary_ppc64le": "bbbd3519d1", + "binary_amd64_compat": "bbbd3519d1", + "binary_amd64_musl": "bbbd3519d1", + "binary_riscv64": "bbbd3519d1", + "binary_s390x": "bbbd3519d1", + "binary_loongarch64": "bbbd3519d1", + "Builds": "f5dffeecb8", + "Install packages (release)": "ba0c89660e", + "Install packages (aarch64)": "ba0c89660e", + "Stateful tests (asan)": "32a9a1aba9", + "Stateful tests (tsan)": "32a9a1aba9", + "Stateful tests (msan)": "32a9a1aba9", + "Stateful tests (ubsan)": "32a9a1aba9", + "Stateful tests (debug)": "32a9a1aba9", + "Stateful tests (release)": "32a9a1aba9", + "Stateful tests (coverage)": "32a9a1aba9", + "Stateful tests (aarch64)": "32a9a1aba9", + "Stateful tests (release, ParallelReplicas)": "32a9a1aba9", + "Stateful tests (debug, ParallelReplicas)": "32a9a1aba9", + "Stateless tests (asan)": "deb6778b88", + "Stateless tests (tsan)": "deb6778b88", + "Stateless tests (msan)": "deb6778b88", + "Stateless tests (ubsan)": "deb6778b88", + "Stateless tests (debug)": "deb6778b88", + "Stateless tests (release)": "deb6778b88", + "Stateless tests (coverage)": "deb6778b88", + "Stateless tests (aarch64)": "deb6778b88", + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)": "deb6778b88", + "Stateless tests (debug, s3 storage)": "deb6778b88", + "Stateless tests (tsan, s3 storage)": "deb6778b88", + "Stress test (debug)": "aa298abf10", + "Stress test (tsan)": "aa298abf10", + "Upgrade check (debug)": "5ce4d3ee02", + "Integration tests (asan, old analyzer)": "42e58be3aa", + "Integration tests (tsan)": "42e58be3aa", + "Integration tests (aarch64)": "42e58be3aa", + "Integration tests flaky check (asan)": "42e58be3aa", + "Compatibility check (release)": "ecb69d8c4b", + "Compatibility check (aarch64)": "ecb69d8c4b", + "Unit tests (release)": "09d00b702e", + "Unit tests (asan)": "09d00b702e", + "Unit tests (msan)": "09d00b702e", + "Unit tests (tsan)": "09d00b702e", + "Unit tests (ubsan)": "09d00b702e", + "AST fuzzer (debug)": "c38ebf947f", + "AST fuzzer (asan)": "c38ebf947f", + "AST fuzzer (msan)": "c38ebf947f", + "AST fuzzer (tsan)": "c38ebf947f", + "AST fuzzer (ubsan)": "c38ebf947f", + "Stateless tests flaky check (asan)": "deb6778b88", + "Performance Comparison (release)": "a8a7179258", + "ClickBench (release)": "45c07c4aa6", + "ClickBench (aarch64)": "45c07c4aa6", + "Docker server image": "6a24d5b187", + "Docker keeper image": "6a24d5b187", + "Docs check": "4764154c62", + "Fast test": "cb269133f2", + "Style check": "ffffffffff", + "Stateful tests (ubsan, ParallelReplicas)": "32a9a1aba9", + "Stress test (msan)": "aa298abf10", + "Upgrade check (asan)": "5ce4d3ee02", + } + ci_cache = CiCache(job_digests=job_digest, cache_enabled=True, s3=S3Helper()) + ci_cache.update() diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 10867ea1444..04fd44a87e9 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -624,7 +624,10 @@ class TestCIConfig(unittest.TestCase): + MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS ) - self.assertTrue(CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, "We must never await on Builds Report") + self.assertTrue( + CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, + "We must never await on Builds Report", + ) self.assertCountEqual( list(ci_cache.jobs_to_wait), MOCK_REQUIRED_BUILDS, From d83428daafc665542023fac0a9add048603ad224 Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 14:52:33 +0200 Subject: [PATCH 65/74] fix in ci unittests --- tests/ci/test_ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 04fd44a87e9..44142050821 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -652,7 +652,7 @@ class TestCIConfig(unittest.TestCase): assert not ci_cache.jobs_to_skip MOCK_AFFECTED_JOBS = [ - CI.JobNames.DOCS_CHECK, + CI.JobNames.FAST_TEST, ] MOCK_REQUIRED_BUILDS = [] From dd9fe61d1a973a7fa528259b507218cf264548fe Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 17:44:32 +0200 Subject: [PATCH 66/74] CI: New Release workflow updates and fixes --- .github/actions/release/action.yml | 27 +++++------- tests/ci/artifactory.py | 56 ++++++++++++++----------- tests/ci/ci_utils.py | 3 +- tests/ci/create_release.py | 66 +++++++++++++++++++----------- 4 files changed, 87 insertions(+), 65 deletions(-) diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml index 99ec02662f6..c3897682a33 100644 --- a/.github/actions/release/action.yml +++ b/.github/actions/release/action.yml @@ -62,8 +62,8 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/create_release.py --set-progress-started --progress "update ChangeLog" - [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 + git checkout master + python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv echo "Update docker version" @@ -96,17 +96,13 @@ runs: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} ### Changelog category (leave one): - Not for changelog (changelog entry is not required) - - name: Reset changes if Dry-run - if: ${{ inputs.dry-run }} + - name: Complete previous steps and Restore git state + if: ${{ inputs.type == 'patch' }} shell: bash run: | - git reset --hard HEAD - - name: Checkout back to GITHUB_REF - shell: bash - run: | - git checkout "$GITHUB_REF_NAME" - # set current progress to OK python3 ./tests/ci/create_release.py --set-progress-completed + git reset --hard HEAD + git checkout "$GITHUB_REF_NAME" - name: Create GH Release shell: bash if: ${{ inputs.type == 'patch' }} @@ -146,24 +142,23 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/create_release.py --set-progress-started --progress "docker server release" cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker server release" export CHECK_NAME="Docker server image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./tests/ci/create_release.py --set-progress-completed + python3 ./create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/create_release.py --set-progress-started --progress "docker keeper release" cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker keeper release" export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./tests/ci/create_release.py --set-progress-completed - - name: Set Release progress completed + python3 ./create_release.py --set-progress-completed + - name: Set current Release progress to Completed with OK shell: bash run: | - # If we here - set completed status, to post proper Slack OK or FAIL message in the next step python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" python3 ./tests/ci/create_release.py --set-progress-completed - name: Post Slack Message diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 98a0345c6bd..86dcaf79854 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -43,7 +43,6 @@ class R2MountPoint: self.bucket_name = self._PROD_BUCKET_NAME self.aux_mount_options = "" - self.async_mount = False if self.app == MountPointApp.S3FS: self.cache_dir = "/home/ubuntu/s3fs_cache" # self.aux_mount_options += "-o nomodtime " if self.NOMODTIME else "" not for s3fs @@ -57,7 +56,6 @@ class R2MountPoint: self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" elif self.app == MountPointApp.RCLONE: # run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return - self.async_mount = True self.cache_dir = "/home/ubuntu/rclone_cache" self.aux_mount_options += "--no-modtime " if self.NOMODTIME else "" self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose @@ -85,10 +83,12 @@ class R2MountPoint: Shell.run(_UNMOUNT_CMD) Shell.run(_MKDIR_CMD) Shell.run(_MKDIR_FOR_CACHE) - # didn't manage to use simple run() and not block or fail - Shell.run_as_daemon(self.mount_cmd) - if self.async_mount: - time.sleep(3) + if self.app == MountPointApp.S3FS: + Shell.run(self.mount_cmd, check=True) + else: + # didn't manage to use simple run() and without blocking or failure + Shell.run_as_daemon(self.mount_cmd) + time.sleep(3) Shell.run(_TEST_MOUNT_CMD, check=True) @classmethod @@ -107,6 +107,7 @@ class DebianArtifactory: _PROD_REPO_URL = "https://packages.clickhouse.com/deb" def __init__(self, release_info: ReleaseInfo, dry_run: bool): + self.release_info = release_info self.codename = release_info.codename self.version = release_info.version if dry_run: @@ -154,9 +155,8 @@ class DebianArtifactory: print("Running test command:") print(f" {cmd}") Shell.run(cmd, check=True) - release_info = ReleaseInfo.from_file() - release_info.debian_command = debian_command - release_info.dump() + self.release_info.debian_command = debian_command + self.release_info.dump() def _copy_if_not_exists(src: Path, dst: Path) -> Path: @@ -177,6 +177,7 @@ class RpmArtifactory: _SIGN_KEY = "885E2BDCF96B0B45ABF058453E4AD4719DDE9A38" def __init__(self, release_info: ReleaseInfo, dry_run: bool): + self.release_info = release_info self.codename = release_info.codename self.version = release_info.version if dry_run: @@ -230,9 +231,8 @@ class RpmArtifactory: print("Running test command:") print(f" {cmd}") Shell.run(cmd, check=True) - release_info = ReleaseInfo.from_file() - release_info.rpm_command = rpm_command - release_info.dump() + self.release_info.rpm_command = rpm_command + self.release_info.dump() class TgzArtifactory: @@ -240,6 +240,7 @@ class TgzArtifactory: _PROD_REPO_URL = "https://packages.clickhouse.com/tgz" def __init__(self, release_info: ReleaseInfo, dry_run: bool): + self.release_info = release_info self.codename = release_info.codename self.version = release_info.version if dry_run: @@ -290,9 +291,8 @@ class TgzArtifactory: expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" Shell.run("rm /tmp/tmp.tgz*") - release_info = ReleaseInfo.from_file() - release_info.tgz_command = cmd - release_info.dump() + self.release_info.tgz_command = cmd + self.release_info.dump() def parse_args() -> argparse.Namespace: @@ -340,9 +340,7 @@ def parse_args() -> argparse.Namespace: if __name__ == "__main__": args = parse_args() - assert args.dry_run - release_info = ReleaseInfo.from_file() """ Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: ERROR : IO error: NotImplemented: versionId not implemented @@ -350,26 +348,38 @@ if __name__ == "__main__": """ mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) if args.export_debian: - with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_DEB) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.EXPORT_DEB + ) as release_info: mp.init() DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() mp.teardown() if args.export_rpm: - with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_RPM) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.EXPORT_RPM + ) as release_info: mp.init() RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() mp.teardown() if args.export_tgz: - with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_TGZ) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.EXPORT_TGZ + ) as release_info: mp.init() TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() mp.teardown() if args.test_debian: - with ReleaseContextManager(release_progress=ReleaseProgress.TEST_DEB) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.TEST_DEB + ) as release_info: DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_tgz: - with ReleaseContextManager(release_progress=ReleaseProgress.TEST_TGZ) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.TEST_TGZ + ) as release_info: TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_rpm: - with ReleaseContextManager(release_progress=ReleaseProgress.TEST_RPM) as _: + with ReleaseContextManager( + release_progress=ReleaseProgress.TEST_RPM + ) as release_info: RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index efbf014cd52..9a1b12af310 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -182,10 +182,11 @@ class Shell: check=False, ) if result.returncode == 0: + print(f"stdout: {result.stdout.strip()}") res = result.stdout else: print( - f"ERROR: stdout {result.stdout.strip()}, stderr {result.stderr.strip()}" + f"ERROR: stdout: {result.stdout.strip()}, stderr: {result.stderr.strip()}" ) if check: assert result.returncode == 0 diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 4347cfebb54..a0b4083b673 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -43,6 +43,7 @@ class ReleaseProgress: TEST_TGZ = "test TGZ packages" TEST_RPM = "test RPM packages" TEST_DEB = "test DEB packages" + COMPLETED = "completed" class ReleaseProgressDescription: @@ -108,6 +109,12 @@ class ReleaseInfo: release_progress: str = "" progress_description: str = "" + def is_patch(self): + return self.release_branch != "master" + + def is_new_release_branch(self): + return self.release_branch == "master" + @staticmethod def from_file() -> "ReleaseInfo": with open(RELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: @@ -126,12 +133,12 @@ class ReleaseInfo: release_tag = None previous_release_tag = None previous_release_sha = None - codename = None + codename = "" assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch Shell.run( - f"git merge-base --is-ancestor origin/{commit_ref} origin/master", + f"git merge-base --is-ancestor {commit_ref} origin/master", check=True, ) with checkout(commit_ref): @@ -146,9 +153,6 @@ class ReleaseInfo: git.latest_tag == expected_prev_tag ), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]" release_tag = version.describe - codename = ( - VersionType.STABLE - ) # dummy value (artifactory won't be updated for new release) previous_release_tag = expected_prev_tag previous_release_sha = Shell.run_strict( f"git rev-parse {previous_release_tag}" @@ -205,7 +209,7 @@ class ReleaseInfo: and commit_sha and release_tag and version - and codename in ("lts", "stable") + and (codename in ("lts", "stable") or release_type == "new") ) self.release_branch = release_branch @@ -320,24 +324,27 @@ class ReleaseInfo: Shell.run( f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" ) - self.version_bump_pr = GHActions.get_pr_url_by_branch( - repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors - ) + self.version_bump_pr = "dry-run" + else: + self.version_bump_pr = GHActions.get_pr_url_by_branch( + repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors + ) def update_release_info(self, dry_run: bool) -> "ReleaseInfo": - branch = f"auto/{release_info.release_tag}" - if not dry_run: - url = GHActions.get_pr_url_by_branch(repo=GITHUB_REPOSITORY, branch=branch) - else: - url = "dry-run" - - print(f"ChangeLog PR url [{url}]") - self.changelog_pr = url - print(f"Release url [{url}]") - self.release_url = ( - f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" - ) - self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.release_branch} clickhouse --version" + if self.release_branch != "master": + branch = f"auto/{release_info.release_tag}" + if not dry_run: + url = GHActions.get_pr_url_by_branch( + repo=GITHUB_REPOSITORY, branch=branch + ) + else: + url = "dry-run" + print(f"ChangeLog PR url [{url}]") + self.changelog_pr = url + print(f"Release url [{url}]") + self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + if self.release_progress == ReleaseProgress.COMPLETED: + self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" self.dump() return self @@ -712,13 +719,22 @@ if __name__ == "__main__": if args.post_status: release_info = ReleaseInfo.from_file() release_info.update_release_info(dry_run=args.dry_run) - if release_info.debian_command: + if release_info.is_new_release_branch(): + title = "New release branch" + else: + title = "New release" + if ( + release_info.progress_description == ReleaseProgressDescription.OK + and release_info.release_progress == ReleaseProgress.COMPLETED + ): + title = "Completed: " + title CIBuddy(dry_run=args.dry_run).post_done( - f"New release issued", dataclasses.asdict(release_info) + title, dataclasses.asdict(release_info) ) else: + title = "Failed: " + title CIBuddy(dry_run=args.dry_run).post_critical( - f"Failed to issue new release", dataclasses.asdict(release_info) + title, dataclasses.asdict(release_info) ) if args.set_progress_started: From 20d4b16fc1d92632b714368c85ac6701723ef096 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 22:19:30 +0200 Subject: [PATCH 67/74] Update 03207_composite_expressions_lambda_consistent_formatting.sql --- .../03207_composite_expressions_lambda_consistent_formatting.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql index 42c823cf476..2e2f5510876 100644 --- a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql +++ b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SELECT [1, (x -> 1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT (1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT map(1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 989476d5234bffd223988bf8aa88e2021e999574 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 06:41:52 +0200 Subject: [PATCH 68/74] Make test `01592_long_window_functions1` lighter --- .../0_stateless/01592_long_window_functions1.sql | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01592_long_window_functions1.sql b/tests/queries/0_stateless/01592_long_window_functions1.sql index d2d32e24eaa..671245599cc 100644 --- a/tests/queries/0_stateless/01592_long_window_functions1.sql +++ b/tests/queries/0_stateless/01592_long_window_functions1.sql @@ -8,14 +8,14 @@ drop table if exists stack; set max_insert_threads = 4; create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64) -Engine = MergeTree -partition by toYYYYMM(dt) +Engine = MergeTree +partition by toYYYYMM(dt) order by (brand_id, toStartOfHour(dt)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -insert into stack -select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100, +insert into stack +select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100, toDateTime('2020-02-01 00:00:00')+number/10, intDiv(number,100)+1 -from numbers_mt(10000000); +from numbers_mt(1000000); select '---- arrays ----'; @@ -32,8 +32,8 @@ select '---- window f ----'; select cityHash64( toString( groupArray (tuple(*) ) )) from ( select brand_id, rack_id, quantity from ( select brand_id, rack_id, quantity, row_number() over (partition by brand_id, rack_id order by quantity) rn - from stack ) as t0 - where rn <= 2 + from stack ) as t0 + where rn <= 2 order by brand_id, rack_id, quantity ) t; From ed02246e0c2019f9ee661e9636f166b955a672de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:46:41 +0200 Subject: [PATCH 69/74] What if I will change this test? --- tests/integration/test_ssl_cert_authentication/test.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 756a1e1996c..3af88759e82 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -43,15 +43,10 @@ def started_cluster(): config = """ - none - + strict {certificateFile} {privateKeyFile} {caConfig} - - - AcceptCertificateHandler - """ From 3d4604a64ad01c70c80924ea11154514bab4e424 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:47:08 +0200 Subject: [PATCH 70/74] Revert "What if I will change this test?" This reverts commit ed02246e0c2019f9ee661e9636f166b955a672de. --- tests/integration/test_ssl_cert_authentication/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 3af88759e82..756a1e1996c 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -43,10 +43,15 @@ def started_cluster(): config = """ - strict + none + {certificateFile} {privateKeyFile} {caConfig} + + + AcceptCertificateHandler + """ From 1a3559fbc3c5257c4e0f5ec16eda3d09d8ebcca0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:46:41 +0200 Subject: [PATCH 71/74] What if I will change this test? --- tests/integration/test_ssl_cert_authentication/test.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 756a1e1996c..3af88759e82 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -43,15 +43,10 @@ def started_cluster(): config = """ - none - + strict {certificateFile} {privateKeyFile} {caConfig} - - - AcceptCertificateHandler - """ From e708219f6aecfe9934827f453665e03142ad5112 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 22 Jul 2024 13:01:27 +0200 Subject: [PATCH 72/74] CI: Print instance info in runner's init script --- tests/ci/worker/init_runner.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index d6cdb6d9c57..1bfeeb38c15 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init v1" +echo "Running init v1.1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner @@ -66,6 +66,11 @@ bash /usr/local/share/scripts/init-network.sh RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text) LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE" export LABELS +echo "Instance Labels: $LABELS" + +LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle) +export LIFE_CYCLE +echo "Instance lifecycle: $LIFE_CYCLE" # Refresh CloudWatch agent config aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json @@ -124,10 +129,6 @@ terminate_decrease_and_exit() { declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh check_spot_instance_is_old() { - # This function should be executed ONLY BETWEEN runnings. - # It's unsafe to execute while the runner is working! - local LIFE_CYCLE - LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle) if [ "$LIFE_CYCLE" == "spot" ]; then local UPTIME UPTIME=$(< /proc/uptime) From cd700c59b1d0b894a7a8461a33d994732b7864f0 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 22 Jul 2024 13:17:25 +0200 Subject: [PATCH 73/74] minor fix --- tests/ci/auto_release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 39ab3156c80..f2386fe207f 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -191,7 +191,7 @@ def main(): title=f"Auto Release Status for {release_info.release_branch}", body=release_info.to_dict(), ) - if args.post_auto_release_complete: + elif args.post_auto_release_complete: assert args.wf_status, "--wf-status Required with --post-auto-release-complete" if args.wf_status != SUCCESS: CIBuddy(dry_run=False).post_job_error( From 69ad8feb9078421aca99709ffb839e2f3b923427 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 22 Jul 2024 13:38:10 +0200 Subject: [PATCH 74/74] add instance type --- tests/ci/worker/init_runner.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 1bfeeb38c15..5177e112edd 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -72,6 +72,9 @@ LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-lif export LIFE_CYCLE echo "Instance lifecycle: $LIFE_CYCLE" +INSTANCE_TYPE=$(ec2metadata --instance-type) +echo "Instance type: $INSTANCE_TYPE" + # Refresh CloudWatch agent config aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json systemctl restart amazon-cloudwatch-agent.service