From 874116a107918a182afd3e5e3c3c9aa2f898cafa Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 14:53:48 +0100 Subject: [PATCH 001/104] impl --- src/Interpreters/ConcurrentHashJoin.cpp | 53 ++++---- src/Interpreters/ConcurrentHashJoin.h | 3 +- src/Interpreters/HashJoin/AddedColumns.cpp | 12 +- src/Interpreters/HashJoin/AddedColumns.h | 17 ++- src/Interpreters/HashJoin/HashJoin.cpp | 95 +++++++++++--- src/Interpreters/HashJoin/HashJoin.h | 132 +++++++++++++++++++ src/Interpreters/HashJoin/HashJoinMethods.h | 135 ++++++++++---------- 7 files changed, 329 insertions(+), 118 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index ac940c62a1a..08f4f422496 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -24,6 +24,8 @@ #include #include +using namespace DB; + namespace ProfileEvents { extern const Event HashJoinPreallocatedElementsInHashTables; @@ -53,6 +55,19 @@ void updateStatistics(const auto & hash_joins, const DB::StatsCollectingParams & DB::getHashTablesStatistics().update(sum_of_sizes, *median_size, params); } +Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) +{ + Blocks inner_blocks; + for (const auto & block : blocks) + { + chassert(!block.wasScattered()); + if (block.wasScattered()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not scattered block is expected here"); + inner_blocks.push_back(*block.block); + } + return concatenateBlocks(inner_blocks); +} + } namespace DB @@ -165,7 +180,7 @@ ConcurrentHashJoin::~ConcurrentHashJoin() bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_limits) { - Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); + auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); size_t blocks_left = 0; for (const auto & block : dispatched_blocks) @@ -193,7 +208,7 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_li bool limit_exceeded = !hash_join->data->addBlockToJoin(dispatched_block, check_limits); - dispatched_block = {}; + dispatched_block = Block{}; blocks_left--; if (limit_exceeded) @@ -209,7 +224,7 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_li void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) { - Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); + auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); block = {}; for (size_t i = 0; i < dispatched_blocks.size(); ++i) { @@ -221,7 +236,7 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); } - block = concatenateBlocks(dispatched_blocks); + block = ::concatenateBlocks(dispatched_blocks); } void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const @@ -300,10 +315,9 @@ static ALWAYS_INLINE IColumn::Selector hashToSelector(const WeakHash32 & hash, s return selector; } -IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_columns_names, const Block & from_block) +IColumn::Selector selectDispatchBlock(size_t num_shards, const Strings & key_columns_names, const Block & from_block) { size_t num_rows = from_block.rows(); - size_t num_shards = hash_joins.size(); WeakHash32 hash(num_rows); for (const auto & key_name : key_columns_names) @@ -315,27 +329,22 @@ IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_co return hashToSelector(hash, num_shards); } -Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +HashJoin::ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) { - /// TODO: use JoinCommon::scatterBlockByHash size_t num_shards = hash_joins.size(); - size_t num_cols = from_block.columns(); - - IColumn::Selector selector = selectDispatchBlock(key_columns_names, from_block); - - Blocks result(num_shards); + IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); + std::vector selectors(num_shards); for (size_t i = 0; i < num_shards; ++i) - result[i] = from_block.cloneEmpty(); - - for (size_t i = 0; i < num_cols; ++i) + selectors[i].reserve(selector.size() / num_shards + 1); + for (size_t i = 0; i < selector.size(); ++i) { - auto dispatched_columns = from_block.getByPosition(i).column->scatter(num_shards, selector); - assert(result.size() == dispatched_columns.size()); - for (size_t block_index = 0; block_index < num_shards; ++block_index) - { - result[block_index].getByPosition(i).column = std::move(dispatched_columns[block_index]); - } + const size_t shard = selector[i]; + selectors[shard].push_back(i); } + HashJoin::ScatteredBlocks result; + result.reserve(num_shards); + for (size_t i = 0; i < num_shards; ++i) + result.emplace_back(from_block, std::move(selectors[i])); return result; } diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index a911edaccc3..8e75bcd874b 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -78,8 +78,7 @@ private: std::mutex totals_mutex; Block totals; - IColumn::Selector selectDispatchBlock(const Strings & key_columns_names, const Block & from_block); - Blocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); + HashJoin::ScatteredBlocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); }; UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression); diff --git a/src/Interpreters/HashJoin/AddedColumns.cpp b/src/Interpreters/HashJoin/AddedColumns.cpp index 930a352744d..45185021ded 100644 --- a/src/Interpreters/HashJoin/AddedColumns.cpp +++ b/src/Interpreters/HashJoin/AddedColumns.cpp @@ -3,14 +3,16 @@ namespace DB { -JoinOnKeyColumns::JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) - : key_names(key_names_) - , materialized_keys_holder(JoinCommon::materializeColumns( - block, key_names)) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. +JoinOnKeyColumns::JoinOnKeyColumns( + const HashJoin::ScatteredBlock & block_, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) + : block(block_) + , key_names(key_names_) + /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. + , materialized_keys_holder(JoinCommon::materializeColumns(*block.block, key_names)) , key_columns(JoinCommon::getRawPointers(materialized_keys_holder)) , null_map(nullptr) , null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map)) - , join_mask_column(JoinCommon::getColumnAsMask(block, cond_column_name)) + , join_mask_column(JoinCommon::getColumnAsMask(*block.block, cond_column_name)) , key_sizes(key_sizes_) { } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 13a7df6f498..daba6c4f2e0 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -14,6 +14,8 @@ using ExpressionActionsPtr = std::shared_ptr; struct JoinOnKeyColumns { + const HashJoin::ScatteredBlock & block; + Names key_names; Columns materialized_keys_holder; @@ -27,9 +29,14 @@ struct JoinOnKeyColumns Sizes key_sizes; - explicit JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_); + JoinOnKeyColumns( + const HashJoin::ScatteredBlock & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_); - bool isRowFiltered(size_t i) const { return join_mask_column.isRowFiltered(i); } + bool isRowFiltered(size_t i) const + { + chassert(std::ranges::find(block.selector, i) != block.selector.end(), fmt::format("Row {} is not in block", i)); + return join_mask_column.isRowFiltered(i); + } }; template @@ -55,7 +62,7 @@ public: }; AddedColumns( - const Block & left_block_, + const HashJoin::ScatteredBlock & left_block_, const Block & block_with_columns_to_add, const Block & saved_block_sample, const HashJoin & join, @@ -63,7 +70,8 @@ public: ExpressionActionsPtr additional_filter_expression_, bool is_asof_join, bool is_join_get_) - : left_block(left_block_) + : src_block(left_block_) + , left_block(*left_block_.block) , join_on_keys(join_on_keys_) , additional_filter_expression(additional_filter_expression_) , rows_to_add(left_block.rows()) @@ -134,6 +142,7 @@ public: const IColumn & leftAsofKey() const { return *left_asof_key; } + const HashJoin::ScatteredBlock & src_block; Block left_block; std::vector join_on_keys; ExpressionActionsPtr additional_filter_expression; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 0c7cad4360d..e14b4d5bb01 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -441,15 +441,21 @@ Block HashJoin::prepareRightBlock(const Block & block) const return prepareRightBlock(block, savedBlockSample()); } -bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) +bool HashJoin::addBlockToJoin(const Block & source_block, bool check_limits) +{ + auto scattered_block = ScatteredBlock{source_block}; + return addBlockToJoin(scattered_block, check_limits); +} + +bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) { if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Join data was released"); /// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency. /// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code. - if (unlikely(source_block_.rows() > std::numeric_limits::max())) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Too many rows in right table block for HashJoin: {}", source_block_.rows()); + if (unlikely(source_block.rows() > std::numeric_limits::max())) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Too many rows in right table block for HashJoin: {}", source_block.rows()); /** We do not allocate memory for stored blocks inside HashJoin, only for hash table. * In case when we have all the blocks allocated before the first `addBlockToJoin` call, will already be quite high. @@ -458,7 +464,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (!memory_usage_before_adding_blocks) memory_usage_before_adding_blocks = getCurrentQueryMemoryUsage(); - Block source_block = source_block_; if (strictness == JoinStrictness::Asof) { chassert(kind == JoinKind::Left || kind == JoinKind::Inner); @@ -485,8 +490,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) for (size_t i = 0; i < asof_column_nullable.size(); ++i) negative_null_map[i] = !asof_column_nullable[i]; - for (auto & column : source_block) - column.column = column.column->filter(negative_null_map, -1); + source_block.filter(negative_null_map); } } } @@ -498,10 +502,11 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) for (const auto & column_name : right_key_names) { const auto & column = source_block.getByName(column_name).column; + /// TODO: do it once for the original block before splitting all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } - Block block_to_save = prepareRightBlock(source_block); + Block block_to_save = prepareRightBlock(*source_block.block); if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); @@ -512,6 +517,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) && (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join) || (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join))) { + chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join if (tmp_stream == nullptr) { tmp_stream = &tmp_data->createStream(right_sample_block); @@ -535,11 +541,14 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) && ((min_bytes_to_compress && getTotalByteCount() >= min_bytes_to_compress) || (min_rows_to_compress && getTotalRowCount() >= min_rows_to_compress))) { + chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join block_to_save = block_to_save.compress(); have_compressed = true; } - data->blocks_allocated_size += block_to_save.allocatedBytes(); + /// In case of scattered block we account proportional share of the source block bytes. + /// For not scattered columns it will be trivial (bytes * N / N) calculation. + data->blocks_allocated_size += block_to_save.rows() ? block_to_save.allocatedBytes() * rows / block_to_save.rows() : 0; data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); @@ -567,7 +576,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) save_nullmap |= (*null_map)[i]; } - auto join_mask_col = JoinCommon::getColumnAsMask(source_block, onexprs[onexpr_idx].condColumnNames().second); + auto join_mask_col = JoinCommon::getColumnAsMask(*source_block.block, onexprs[onexpr_idx].condColumnNames().second); /// Save blocks that do not hold conditions in ON section ColumnUInt8::MutablePtr not_joined_map = nullptr; if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData()) @@ -592,27 +601,31 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) bool is_inserted = false; if (kind != JoinKind::Cross) { - joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) - { - size_t size = HashJoinMethods>::insertFromBlockImpl( + joinDispatch( + kind, + strictness, + data->maps[onexpr_idx], + [&](auto kind_, auto strictness_, auto & map) + { + size_t size = HashJoinMethods>::insertFromBlockImpl( *this, data->type, map, - rows, key_columns, key_sizes[onexpr_idx], stored_block, + source_block.selector, null_map, join_mask_col.getData(), data->pool, is_inserted); - if (flag_per_row) - used_flags->reinit(stored_block); - else if (is_inserted) - /// Number of buckets + 1 value from zero storage - used_flags->reinit(size + 1); - }); + if (flag_per_row) + used_flags->reinit(stored_block); + else if (is_inserted) + /// Number of buckets + 1 value from zero storage + used_flags->reinit(size + 1); + }); } if (!flag_per_row && save_nullmap && is_inserted) @@ -933,6 +946,50 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) } } +void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) +{ + if (!data) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); + + for (const auto & onexpr : table_join->getClauses()) + { + auto cond_column_name = onexpr.condColumnNames(); + JoinCommon::checkTypesOfKeys( + *block.block, + onexpr.key_names_left, + cond_column_name.first, + right_sample_block, + onexpr.key_names_right, + cond_column_name.second); + } + + chassert(kind == JoinKind::Left || kind == JoinKind::Inner); + + std::vectormaps[0])> *> maps_vector; + for (size_t i = 0; i < table_join->getClauses().size(); ++i) + maps_vector.push_back(&data->maps[i]); + + if (joinDispatch( + kind, + strictness, + maps_vector, + [&](auto kind_, auto strictness_, auto & maps_vector_) + { + using MapType = typename MapGetter::Map; + ScatteredBlock remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + if (remaining_block.rows()) + not_processed = std::make_shared(ExtraBlock{std::move(*remaining_block.block)}); + else + not_processed.reset(); + })) + { + /// Joined + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); +} + HashJoin::~HashJoin() { if (!data) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 0b115b9fdbb..008a1425c4e 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -29,6 +29,8 @@ #include #include +#include + namespace DB { @@ -137,11 +139,138 @@ public: return std::make_shared(table_join_, right_sample_block_, any_take_last_row, reserve_num, instance_id); } + struct ScatteredBlock : private boost::noncopyable + { + BlockPtr block; // TODO: we don't need shared_ptr here since if any changes are made to block, they're supposed to be private + IColumn::Selector selector; + + ScatteredBlock(const Block & block_) : block(std::make_shared(block_)), selector(createTrivialSelector(block->rows())) { } + + ScatteredBlock(const Block & block_, IColumn::Selector && selector_) + : block(std::make_shared(block_)), selector(std::move(selector_)) + { + } + + ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) + { + other.block = nullptr; + other.selector.clear(); + } + + ScatteredBlock & operator=(ScatteredBlock && other) noexcept + { + if (this != &other) + { + block = std::move(other.block); + selector = std::move(other.selector); + + other.block = nullptr; + other.selector.clear(); + } + return *this; + } + + operator bool() const { return block && *block; } + + /// Accounts only selected rows + size_t rows() const { return selector.size(); } + + /// Whether block was scattered, i.e. has non-trivial selector + bool wasScattered() const + { + chassert(block); + return selector.size() != block->rows(); + } + + const ColumnWithTypeAndName & getByName(const std::string & name) const + { + chassert(block); + return block->getByName(name); + } + + /// Filters selector by mask discarding rows for which filter is false + void filter(const IColumn::Filter & filter) + { + chassert(block && block->rows() == filter.size()); + auto it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); + selector.resize(std::distance(selector.begin(), it)); + } + + /// Applies selector to block in place + void filterBySelector() + { + chassert(block); + auto columns = block->getColumns(); + for (auto & col : columns) + { + auto c = col->cloneEmpty(); + c->reserve(selector.size()); + /// TODO: create new method in IColumnHelper to devirtualize + for (const auto idx : selector) + c->insertFrom(*col, idx); + col = std::move(c); + } + + *this = ScatteredBlock{block->cloneWithColumns(std::move(columns))}; + } + + /// Cut first num_rows rows from block in place and returns block with remaining rows + ScatteredBlock cut(size_t num_rows) + { + SCOPE_EXIT(filterBySelector()); + + if (num_rows >= rows()) + return Block{}; + + chassert(block); + + IColumn::Selector remaining_selector(selector.begin() + num_rows, selector.end()); + auto remaining = ScatteredBlock{*block, std::move(remaining_selector)}; + + selector.erase(selector.begin() + num_rows, selector.end()); + + return remaining; + } + + void replicate(const IColumn::Offsets & offsets, size_t existing_columns, const std::vector & right_keys_to_replicate) + { + chassert(block); + chassert(offsets.size() == rows()); + + auto columns = block->getColumns(); + for (size_t i = 0; i < existing_columns; ++i) + { + auto c = columns[i]->replicate(offsets); + columns[i] = std::move(c); + } + for (size_t pos : right_keys_to_replicate) + { + auto c = columns[pos]->replicate(offsets); + columns[pos] = std::move(c); + } + + *this = ScatteredBlock{block->cloneWithColumns(std::move(columns))}; + } + + // private: + IColumn::Selector createTrivialSelector(size_t size) + { + IColumn::Selector res(size); + std::iota(res.begin(), res.end(), 0); + return res; + } + }; + + using ScatteredBlocks = std::vector; + /** Add block of data from right hand of JOIN to the map. * Returns false, if some limit was exceeded and you should not insert more data. */ bool addBlockToJoin(const Block & source_block_, bool check_limits) override; + /// Called directly from ConcurrentJoin::addBlockToJoin + bool addBlockToJoin(ScatteredBlock & source_block_, bool check_limits); + void checkTypesOfKeys(const Block & block) const override; /** Join data from the map (that was previously built by calls to addBlockToJoin) to the block with data from "left" table. @@ -149,6 +278,9 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; + /// Called directly from ConcurrentJoin::joinBlock + void joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed); + /// Check joinGet arguments and infer the return type. DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 0dfafa94efc..317dde83314 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -74,10 +74,10 @@ public: HashJoin & join, HashJoin::Type type, MapsTemplate & maps, - size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, Block * stored_block, + const IColumn::Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, @@ -92,14 +92,15 @@ public: is_inserted = true; return 0; - #define M(TYPE) \ - case HashJoin::Type::TYPE: \ - return insertFromBlockImplTypeCase>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ - break; +#define M(TYPE) \ + case HashJoin::Type::TYPE: \ + return insertFromBlockImplTypeCase< \ + typename KeyGetterForType>::Type>( \ + join, *maps.TYPE, key_columns, key_sizes, stored_block, selector, null_map, join_mask, pool, is_inserted); \ + break; - APPLY_FOR_JOIN_VARIANTS(M) - #undef M + APPLY_FOR_JOIN_VARIANTS(M) +#undef M } } @@ -111,6 +112,21 @@ public: const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get = false) + { + HashJoin::ScatteredBlock scattered_block{block}; + auto ret = joinBlockImpl(join, scattered_block, block_with_columns_to_add, maps_, is_join_get); + ret.filterBySelector(); + scattered_block.filterBySelector(); + block = std::move(*scattered_block.block); + return *ret.block; + } + + static HashJoin::ScatteredBlock joinBlockImpl( + const HashJoin & join, + HashJoin::ScatteredBlock & block, + const Block & block_with_columns_to_add, + const MapsTemplateVector & maps_, + bool is_join_get = false) { constexpr JoinFeatures join_features; @@ -121,7 +137,7 @@ public: const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - size_t existing_columns = block.columns(); + size_t existing_columns = block.block->columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -129,7 +145,8 @@ public: */ if constexpr (join_features.right || join_features.full) { - materializeBlockInplace(block); + /// TODO: do materialization once before scattering the source block by hash + materializeBlockInplace(*block.block); } /** For LEFT/INNER JOIN, the saved blocks do not contain keys. @@ -155,23 +172,24 @@ public: else added_columns.reserve(join_features.need_replication); - size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); + const size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); /// Do not hold memory for join_on_keys anymore added_columns.join_on_keys.clear(); - Block remaining_block = sliceBlock(block, num_joined); - + auto remaining_block = block.cut(num_joined); added_columns.buildOutput(); + + if constexpr (join_features.need_filter) + block.filter(added_columns.filter); + + block.filterBySelector(); + for (size_t i = 0; i < added_columns.size(); ++i) - block.insert(added_columns.moveColumn(i)); + block.block->insert(added_columns.moveColumn(i)); std::vector right_keys_to_replicate [[maybe_unused]]; if constexpr (join_features.need_filter) { - /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); - /// Add join key columns from right block if needed using value from left table because of equality for (size_t i = 0; i < join.required_right_keys.columns(); ++i) { @@ -183,7 +201,7 @@ public: const auto & left_column = block.getByName(join.required_right_keys_sources[i]); const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); - block.insert(std::move(right_col)); + block.block->insert(std::move(right_col)); } } else if (has_required_right_keys) @@ -199,28 +217,17 @@ public: const auto & left_column = block.getByName(join.required_right_keys_sources[i]); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); - block.insert(std::move(right_col)); + block.block->insert(std::move(right_col)); if constexpr (join_features.need_replication) - right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); + right_keys_to_replicate.push_back(block.block->getPositionByName(right_col_name)); } } if constexpr (join_features.need_replication) { std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; - - /// If ALL ... JOIN - we replicate all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - { - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); - } - - /// Replicate additional right keys - for (size_t pos : right_keys_to_replicate) - { - block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); - } + block.replicate(*offsets_to_replicate, existing_columns, right_keys_to_replicate); } return remaining_block; @@ -244,8 +251,16 @@ private: template static size_t NO_INLINE insertFromBlockImplTypeCase( - HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) + HashJoin & join, + HashMap & map, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + const IColumn::Selector & selector, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted) { [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; @@ -259,9 +274,10 @@ private: /// For ALL and ASOF join always insert values is_inserted = !mapped_one || is_asof_join; - for (size_t i = 0; i < rows; ++i) + for (size_t ind : selector) { - if (null_map && (*null_map)[i]) + chassert(!null_map || ind < null_map->size()); + if (null_map && (*null_map)[ind]) { /// nulls are not inserted into hash table, /// keep them for RIGHT and FULL joins @@ -270,15 +286,16 @@ private: } /// Check condition for right table from ON section - if (join_mask && !(*join_mask)[i]) + chassert(!join_mask || ind < join_mask->size()); + if (join_mask && !(*join_mask)[ind]) continue; if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + Inserter::insertAsof(join, map, key_getter, stored_block, ind, pool, *asof_column); else if constexpr (mapped_one) - is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, ind, pool); else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + Inserter::insertAll(join, map, key_getter, stored_block, ind, pool); } return map.getBufferSizeInCells(); } @@ -396,7 +413,8 @@ private: { constexpr JoinFeatures join_features; - size_t rows = added_columns.rows_to_add; + auto & block = added_columns.src_block; + size_t rows = block.rows(); if constexpr (need_filter) added_columns.filter = IColumn::Filter(rows, 0); @@ -410,12 +428,14 @@ private: size_t i = 0; for (; i < rows; ++i) { + const auto ind = block.selector[i]; + if constexpr (join_features.need_replication) { if (unlikely(current_offset >= max_joined_block_rows)) { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); + added_columns.offsets_to_replicate->resize_assume_reserved(ind); + added_columns.filter.resize_assume_reserved(ind); break; } } @@ -426,12 +446,12 @@ private: for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) { const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; - if (join_keys.null_map && (*join_keys.null_map)[i]) - continue; + if (join_keys.null_map && (*join_keys.null_map)[ind]) + continue; - bool row_acceptable = !join_keys.isRowFiltered(i); + bool row_acceptable = !join_keys.isRowFiltered(ind); using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), ind, pool) : FindResult(); if (find_result.isFound()) { @@ -441,7 +461,7 @@ private: { const IColumn & left_asof_key = added_columns.leftAsofKey(); - auto row_ref = mapped->findAsof(left_asof_key, i); + auto row_ref = mapped->findAsof(left_asof_key, ind); if (row_ref.block) { setUsed(added_columns.filter, i); @@ -834,23 +854,6 @@ private: return left_row_iter; } - /// Cut first num_rows rows from block in place and returns block with remaining rows - static Block sliceBlock(Block & block, size_t num_rows) - { - size_t total_rows = block.rows(); - if (num_rows >= total_rows) - return {}; - size_t remaining_rows = total_rows - num_rows; - Block remaining_block = block.cloneEmpty(); - for (size_t i = 0; i < block.columns(); ++i) - { - auto & col = block.getByPosition(i); - remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); - col.column = col.column->cut(0, num_rows); - } - return remaining_block; - } - /** Since we do not store right key columns, * this function is used to copy left key columns to right key columns. * If the user requests some right columns, we just copy left key columns to right, since they are equal. From 54dd6aa7ee9fd11dbc981f48c40836e2edbaf7c9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 14:57:24 +0100 Subject: [PATCH 002/104] stash --- src/Interpreters/HashJoin/HashJoin.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 008a1425c4e..91239d1fa14 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -252,7 +252,7 @@ public: *this = ScatteredBlock{block->cloneWithColumns(std::move(columns))}; } - // private: + private: IColumn::Selector createTrivialSelector(size_t size) { IColumn::Selector res(size); From 2850f7aaa3cb5f60eecc4bf0452359cd962a4ee4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 17:58:22 +0100 Subject: [PATCH 003/104] stash 2 --- src/Interpreters/ConcurrentHashJoin.cpp | 6 +-- src/Interpreters/HashJoin/AddedColumns.cpp | 4 +- src/Interpreters/HashJoin/AddedColumns.h | 4 +- src/Interpreters/HashJoin/HashJoin.cpp | 10 ++-- src/Interpreters/HashJoin/HashJoin.h | 55 +++++++++++++-------- src/Interpreters/HashJoin/HashJoinMethods.h | 19 +++---- 6 files changed, 55 insertions(+), 43 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 08f4f422496..add0b08666a 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -60,10 +60,8 @@ Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) Blocks inner_blocks; for (const auto & block : blocks) { - chassert(!block.wasScattered()); - if (block.wasScattered()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not scattered block is expected here"); - inner_blocks.push_back(*block.block); + chassert(!block.wasScattered(), "Not scattered block is expected here"); + inner_blocks.push_back(block.getSourceBlock()); } return concatenateBlocks(inner_blocks); } diff --git a/src/Interpreters/HashJoin/AddedColumns.cpp b/src/Interpreters/HashJoin/AddedColumns.cpp index 45185021ded..78b8602ce0e 100644 --- a/src/Interpreters/HashJoin/AddedColumns.cpp +++ b/src/Interpreters/HashJoin/AddedColumns.cpp @@ -8,11 +8,11 @@ JoinOnKeyColumns::JoinOnKeyColumns( : block(block_) , key_names(key_names_) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. - , materialized_keys_holder(JoinCommon::materializeColumns(*block.block, key_names)) + , materialized_keys_holder(JoinCommon::materializeColumns(block.getSourceBlock(), key_names)) , key_columns(JoinCommon::getRawPointers(materialized_keys_holder)) , null_map(nullptr) , null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map)) - , join_mask_column(JoinCommon::getColumnAsMask(*block.block, cond_column_name)) + , join_mask_column(JoinCommon::getColumnAsMask(block.getSourceBlock(), cond_column_name)) , key_sizes(key_sizes_) { } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index daba6c4f2e0..fb6e32efbb8 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -34,7 +34,7 @@ struct JoinOnKeyColumns bool isRowFiltered(size_t i) const { - chassert(std::ranges::find(block.selector, i) != block.selector.end(), fmt::format("Row {} is not in block", i)); + chassert(std::ranges::find(block.getSelector(), i) != block.getSelector().end(), fmt::format("Row {} is not in block", i)); return join_mask_column.isRowFiltered(i); } }; @@ -71,7 +71,7 @@ public: bool is_asof_join, bool is_join_get_) : src_block(left_block_) - , left_block(*left_block_.block) + , left_block(left_block_.getSourceBlock()) , join_on_keys(join_on_keys_) , additional_filter_expression(additional_filter_expression_) , rows_to_add(left_block.rows()) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index e14b4d5bb01..b45dc3a08ef 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -506,7 +506,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } - Block block_to_save = prepareRightBlock(*source_block.block); + Block block_to_save = prepareRightBlock(source_block.getSourceBlock()); if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); @@ -576,7 +576,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) save_nullmap |= (*null_map)[i]; } - auto join_mask_col = JoinCommon::getColumnAsMask(*source_block.block, onexprs[onexpr_idx].condColumnNames().second); + auto join_mask_col = JoinCommon::getColumnAsMask(source_block.getSourceBlock(), onexprs[onexpr_idx].condColumnNames().second); /// Save blocks that do not hold conditions in ON section ColumnUInt8::MutablePtr not_joined_map = nullptr; if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData()) @@ -614,7 +614,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) key_columns, key_sizes[onexpr_idx], stored_block, - source_block.selector, + source_block.getSelector(), null_map, join_mask_col.getData(), data->pool, @@ -955,7 +955,7 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) { auto cond_column_name = onexpr.condColumnNames(); JoinCommon::checkTypesOfKeys( - *block.block, + block.getSourceBlock(), onexpr.key_names_left, cond_column_name.first, right_sample_block, @@ -979,7 +979,7 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) ScatteredBlock remaining_block = HashJoinMethods::joinBlockImpl( *this, block, sample_block_with_columns_to_add, maps_vector_); if (remaining_block.rows()) - not_processed = std::make_shared(ExtraBlock{std::move(*remaining_block.block)}); + not_processed = std::make_shared(std::move(remaining_block).getSourceBlock()); else not_processed.reset(); })) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 91239d1fa14..85137c390f0 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -1,9 +1,10 @@ #pragma once -#include -#include -#include #include +#include +#include +#include +#include #include #include @@ -29,6 +30,7 @@ #include #include +#include <__ranges/ref_view.h> #include namespace DB @@ -141,19 +143,17 @@ public: struct ScatteredBlock : private boost::noncopyable { - BlockPtr block; // TODO: we don't need shared_ptr here since if any changes are made to block, they're supposed to be private - IColumn::Selector selector; + private: + Block block; // TODO: we don't need shared_ptr here since if any changes are made to block, they're supposed to be private - ScatteredBlock(const Block & block_) : block(std::make_shared(block_)), selector(createTrivialSelector(block->rows())) { } + public: + ScatteredBlock(const Block & block_) : block(block_), selector(createTrivialSelector(block.rows())) { } - ScatteredBlock(const Block & block_, IColumn::Selector && selector_) - : block(std::make_shared(block_)), selector(std::move(selector_)) - { - } + ScatteredBlock(const Block & block_, IColumn::Selector && selector_) : block(block_), selector(std::move(selector_)) { } ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) { - other.block = nullptr; + other.block.clear(); other.selector.clear(); } @@ -164,13 +164,20 @@ public: block = std::move(other.block); selector = std::move(other.selector); - other.block = nullptr; + other.block.clear(); other.selector.clear(); } return *this; } - operator bool() const { return block && *block; } + Block & getSourceBlock() & { return block; } + const Block & getSourceBlock() const & { return block; } + + Block && getSourceBlock() && { return std::move(block); } + + const auto & getSelector() const { return selector; } + + operator bool() const { return !!block; } /// Accounts only selected rows size_t rows() const { return selector.size(); } @@ -179,19 +186,19 @@ public: bool wasScattered() const { chassert(block); - return selector.size() != block->rows(); + return selector.size() != block.rows(); } const ColumnWithTypeAndName & getByName(const std::string & name) const { chassert(block); - return block->getByName(name); + return block.getByName(name); } /// Filters selector by mask discarding rows for which filter is false void filter(const IColumn::Filter & filter) { - chassert(block && block->rows() == filter.size()); + chassert(block && block.rows() == filter.size()); auto it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); selector.resize(std::distance(selector.begin(), it)); } @@ -200,7 +207,7 @@ public: void filterBySelector() { chassert(block); - auto columns = block->getColumns(); + auto columns = block.getColumns(); for (auto & col : columns) { auto c = col->cloneEmpty(); @@ -211,7 +218,9 @@ public: col = std::move(c); } - *this = ScatteredBlock{block->cloneWithColumns(std::move(columns))}; + /// We have to to id that way because references to the block should remain valid + block.setColumns(std::move(columns)); + selector = createTrivialSelector(block.rows()); } /// Cut first num_rows rows from block in place and returns block with remaining rows @@ -225,7 +234,7 @@ public: chassert(block); IColumn::Selector remaining_selector(selector.begin() + num_rows, selector.end()); - auto remaining = ScatteredBlock{*block, std::move(remaining_selector)}; + auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; selector.erase(selector.begin() + num_rows, selector.end()); @@ -237,7 +246,7 @@ public: chassert(block); chassert(offsets.size() == rows()); - auto columns = block->getColumns(); + auto columns = block.getColumns(); for (size_t i = 0; i < existing_columns; ++i) { auto c = columns[i]->replicate(offsets); @@ -249,10 +258,14 @@ public: columns[pos] = std::move(c); } - *this = ScatteredBlock{block->cloneWithColumns(std::move(columns))}; + /// We have to to id that way because references to the block should remain valid + block.setColumns(std::move(columns)); + selector = createTrivialSelector(block.rows()); } private: + IColumn::Selector selector; + IColumn::Selector createTrivialSelector(size_t size) { IColumn::Selector res(size); diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 317dde83314..c6ce10b5bad 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -117,8 +117,8 @@ public: auto ret = joinBlockImpl(join, scattered_block, block_with_columns_to_add, maps_, is_join_get); ret.filterBySelector(); scattered_block.filterBySelector(); - block = std::move(*scattered_block.block); - return *ret.block; + block = std::move(scattered_block.getSourceBlock()); + return ret.getSourceBlock(); } static HashJoin::ScatteredBlock joinBlockImpl( @@ -137,7 +137,8 @@ public: const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - size_t existing_columns = block.block->columns(); + auto & source_block = block.getSourceBlock(); + size_t existing_columns = source_block.columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -146,7 +147,7 @@ public: if constexpr (join_features.right || join_features.full) { /// TODO: do materialization once before scattering the source block by hash - materializeBlockInplace(*block.block); + materializeBlockInplace(source_block); } /** For LEFT/INNER JOIN, the saved blocks do not contain keys. @@ -184,7 +185,7 @@ public: block.filterBySelector(); for (size_t i = 0; i < added_columns.size(); ++i) - block.block->insert(added_columns.moveColumn(i)); + source_block.insert(added_columns.moveColumn(i)); std::vector right_keys_to_replicate [[maybe_unused]]; @@ -201,7 +202,7 @@ public: const auto & left_column = block.getByName(join.required_right_keys_sources[i]); const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); - block.block->insert(std::move(right_col)); + source_block.insert(std::move(right_col)); } } else if (has_required_right_keys) @@ -217,10 +218,10 @@ public: const auto & left_column = block.getByName(join.required_right_keys_sources[i]); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); - block.block->insert(std::move(right_col)); + source_block.insert(std::move(right_col)); if constexpr (join_features.need_replication) - right_keys_to_replicate.push_back(block.block->getPositionByName(right_col_name)); + right_keys_to_replicate.push_back(source_block.getPositionByName(right_col_name)); } } @@ -428,7 +429,7 @@ private: size_t i = 0; for (; i < rows; ++i) { - const auto ind = block.selector[i]; + const auto ind = block.getSelector()[i]; if constexpr (join_features.need_replication) { From a4ec9d074f6384b3c7022b0e3f824aad19bb7de1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 22:55:05 +0100 Subject: [PATCH 004/104] stash 3 --- src/Interpreters/ConcurrentHashJoin.cpp | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index add0b08666a..25681999d01 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -66,6 +66,27 @@ Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) return concatenateBlocks(inner_blocks); } +Block materializeColumnsFromSampleBlock(const Block & block_, const Block & sample_block) +{ + Block block = block_; + for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) + { + auto & column = block.getByName(sample_column.name); + + /// There's no optimization for right side const columns. Remove constness if any. + column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); + + if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) + { + column.column = column.column->convertToFullColumnIfLowCardinality(); + column.type = removeLowCardinality(column.type); + } + + if (sample_column.column->isNullable()) + JoinCommon::convertColumnToNullable(column); + } + return block; +} } namespace DB @@ -176,8 +197,10 @@ ConcurrentHashJoin::~ConcurrentHashJoin() } } -bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_limits) +bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_limits) { + /// We prematurely materialize columns here to avoid materializing columns multiple times on each thread. + Block right_block = materializeColumnsFromSampleBlock(right_block_, hash_joins[0]->data->savedBlockSample()); auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); size_t blocks_left = 0; From 6a099996d2d62b4d7ad05c9855cdb04ebd4f7705 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 23:16:25 +0100 Subject: [PATCH 005/104] stash 4 --- src/Interpreters/ConcurrentHashJoin.cpp | 13 +++++++++++-- src/Interpreters/HashJoin/HashJoin.cpp | 1 - src/Interpreters/HashJoin/HashJoin.h | 9 +++------ 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 25681999d01..1394bfaf323 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -66,9 +66,10 @@ Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) return concatenateBlocks(inner_blocks); } -Block materializeColumnsFromSampleBlock(const Block & block_, const Block & sample_block) +Block materializeColumnsFromSampleBlock(const Block & block_, const Block & sample_block, const Names & right_key_names) { Block block = block_; + for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) { auto & column = block.getByName(sample_column.name); @@ -85,6 +86,13 @@ Block materializeColumnsFromSampleBlock(const Block & block_, const Block & samp if (sample_column.column->isNullable()) JoinCommon::convertColumnToNullable(column); } + + for (const auto & column_name : right_key_names) + { + auto & column = block.getByName(column_name).column; + column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); + } + return block; } } @@ -200,7 +208,8 @@ ConcurrentHashJoin::~ConcurrentHashJoin() bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_limits) { /// We prematurely materialize columns here to avoid materializing columns multiple times on each thread. - Block right_block = materializeColumnsFromSampleBlock(right_block_, hash_joins[0]->data->savedBlockSample()); + Block right_block = materializeColumnsFromSampleBlock( + right_block_, hash_joins[0]->data->savedBlockSample(), table_join->getAllNames(JoinTableSide::Right)); auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); size_t blocks_left = 0; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index b45dc3a08ef..8a9089f376a 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -502,7 +502,6 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) for (const auto & column_name : right_key_names) { const auto & column = source_block.getByName(column_name).column; - /// TODO: do it once for the original block before splitting all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 85137c390f0..a8747ba8f3b 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -143,10 +143,6 @@ public: struct ScatteredBlock : private boost::noncopyable { - private: - Block block; // TODO: we don't need shared_ptr here since if any changes are made to block, they're supposed to be private - - public: ScatteredBlock(const Block & block_) : block(block_), selector(createTrivialSelector(block.rows())) { } ScatteredBlock(const Block & block_, IColumn::Selector && selector_) : block(block_), selector(std::move(selector_)) { } @@ -264,14 +260,15 @@ public: } private: - IColumn::Selector selector; - IColumn::Selector createTrivialSelector(size_t size) { IColumn::Selector res(size); std::iota(res.begin(), res.end(), 0); return res; } + + Block block; + IColumn::Selector selector; }; using ScatteredBlocks = std::vector; From 2fb3ec7abe4dc7883e1540166737c818bc0e458d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 Jul 2024 23:41:24 +0100 Subject: [PATCH 006/104] stash 5 --- src/Interpreters/ConcurrentHashJoin.cpp | 5 ++++- src/Interpreters/HashJoin/HashJoinMethods.h | 1 - 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 1394bfaf323..7fad00bbd32 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -210,8 +210,8 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l /// We prematurely materialize columns here to avoid materializing columns multiple times on each thread. Block right_block = materializeColumnsFromSampleBlock( right_block_, hash_joins[0]->data->savedBlockSample(), table_join->getAllNames(JoinTableSide::Right)); - auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); + auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); size_t blocks_left = 0; for (const auto & block : dispatched_blocks) { @@ -254,6 +254,9 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) { + if (hash_joins[0]->data->getKind() == JoinKind::Right || hash_joins[0]->data->getKind() == JoinKind::Full) + materializeBlockInplace(block); + auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); block = {}; for (size_t i = 0; i < dispatched_blocks.size(); ++i) diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index c6ce10b5bad..5acf0c51d3d 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -146,7 +146,6 @@ public: */ if constexpr (join_features.right || join_features.full) { - /// TODO: do materialization once before scattering the source block by hash materializeBlockInplace(source_block); } From 220d32039c028affc1c7378a971529a6e839e811 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jul 2024 22:16:37 +0100 Subject: [PATCH 007/104] stash 6 --- src/Interpreters/HashJoin/HashJoin.cpp | 66 ++++++++++++++++++-------- src/Interpreters/HashJoin/HashJoin.h | 3 +- 2 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 8a9089f376a..ddc6abe9d0d 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -77,6 +77,44 @@ Int64 getCurrentQueryMemoryUsage() return 0; } +Block filterColumnsPresentInSampleBlock(const Block & block, const Block & sample_block) +{ + Block filtered_block; + for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) + { + ColumnWithTypeAndName column = block.getByName(sample_column.name); + filtered_block.insert(std::move(column)); + } + return filtered_block; +} + +Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, const Names & right_key_names) +{ + for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) + { + auto & column = block.getByName(sample_column.name); + + /// There's no optimization for right side const columns. Remove constness if any. + column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); + + if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) + { + column.column = column.column->convertToFullColumnIfLowCardinality(); + column.type = removeLowCardinality(column.type); + } + + if (sample_column.column->isNullable()) + JoinCommon::convertColumnToNullable(column); + } + + for (const auto & column_name : right_key_names) + { + auto & column = block.getByName(column_name).column; + column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); + } + + return block; +} } static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) @@ -411,29 +449,15 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) } } +Block HashJoin::materializeColumnsFromRightBlock(Block block) const +{ + return DB::materializeColumnsFromRightBlock(std::move(block), savedBlockSample(), table_join->getAllNames(JoinTableSide::Right)); +} + Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block_sample_) { - Block structured_block; - for (const auto & sample_column : saved_block_sample_.getColumnsWithTypeAndName()) - { - ColumnWithTypeAndName column = block.getByName(sample_column.name); - - /// There's no optimization for right side const columns. Remove constness if any. - column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); - - if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) - { - column.column = column.column->convertToFullColumnIfLowCardinality(); - column.type = removeLowCardinality(column.type); - } - - if (sample_column.column->isNullable()) - JoinCommon::convertColumnToNullable(column); - - structured_block.insert(std::move(column)); - } - - return structured_block; + Block structured_block = DB::materializeColumnsFromRightBlock(block, saved_block_sample_, {}); + return filterColumnsPresentInSampleBlock(structured_block, saved_block_sample_); } Block HashJoin::prepareRightBlock(const Block & block) const diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index a8747ba8f3b..8d4513e4349 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -518,6 +518,8 @@ public: void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } + Block materializeColumnsFromRightBlock(Block block) const; + private: friend class NotJoinedHash; @@ -596,5 +598,4 @@ private: void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; }; - } From 33af77cda79a595eea4610cc879cdae193a253ec Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jul 2024 22:20:42 +0100 Subject: [PATCH 008/104] stash 7 --- src/Interpreters/ConcurrentHashJoin.cpp | 32 +------------------------ 1 file changed, 1 insertion(+), 31 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 7fad00bbd32..772e14a9867 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -66,35 +66,6 @@ Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) return concatenateBlocks(inner_blocks); } -Block materializeColumnsFromSampleBlock(const Block & block_, const Block & sample_block, const Names & right_key_names) -{ - Block block = block_; - - for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) - { - auto & column = block.getByName(sample_column.name); - - /// There's no optimization for right side const columns. Remove constness if any. - column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); - - if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) - { - column.column = column.column->convertToFullColumnIfLowCardinality(); - column.type = removeLowCardinality(column.type); - } - - if (sample_column.column->isNullable()) - JoinCommon::convertColumnToNullable(column); - } - - for (const auto & column_name : right_key_names) - { - auto & column = block.getByName(column_name).column; - column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); - } - - return block; -} } namespace DB @@ -208,8 +179,7 @@ ConcurrentHashJoin::~ConcurrentHashJoin() bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_limits) { /// We prematurely materialize columns here to avoid materializing columns multiple times on each thread. - Block right_block = materializeColumnsFromSampleBlock( - right_block_, hash_joins[0]->data->savedBlockSample(), table_join->getAllNames(JoinTableSide::Right)); + Block right_block = hash_joins[0]->data->materializeColumnsFromRightBlock(right_block_); auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); size_t blocks_left = 0; From ffd1a6c3a54ca5f223a5559cc9333b446e7d7f04 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jul 2024 23:00:47 +0100 Subject: [PATCH 009/104] stash 8 --- src/Interpreters/HashJoin/HashJoin.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index ddc6abe9d0d..1c08a951c6c 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -88,7 +88,7 @@ Block filterColumnsPresentInSampleBlock(const Block & block, const Block & sampl return filtered_block; } -Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, const Names & right_key_names) +Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, const Names &) { for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) { @@ -107,11 +107,11 @@ Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, JoinCommon::convertColumnToNullable(column); } - for (const auto & column_name : right_key_names) - { - auto & column = block.getByName(column_name).column; - column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); - } + // for (const auto & column_name : right_key_names) + // { + // auto & column = block.getByName(column_name).column; + // column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); + // } return block; } @@ -467,7 +467,8 @@ Block HashJoin::prepareRightBlock(const Block & block) const bool HashJoin::addBlockToJoin(const Block & source_block, bool check_limits) { - auto scattered_block = ScatteredBlock{source_block}; + auto materialized = materializeColumnsFromRightBlock(source_block); + auto scattered_block = ScatteredBlock{materialized}; return addBlockToJoin(scattered_block, check_limits); } @@ -529,7 +530,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } - Block block_to_save = prepareRightBlock(source_block.getSourceBlock()); + Block block_to_save = filterColumnsPresentInSampleBlock(source_block.getSourceBlock(), savedBlockSample()); if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); From 3ca66293cab71fbc76396dcfafa5592181246538 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Aug 2024 22:58:53 +0100 Subject: [PATCH 010/104] fix --- src/Interpreters/HashJoin/HashJoin.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 8d4513e4349..96a325ec367 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -225,7 +225,7 @@ public: SCOPE_EXIT(filterBySelector()); if (num_rows >= rows()) - return Block{}; + return block.cloneEmpty(); chassert(block); From cc379ce941c0e172541b40675d5fefcead73d255 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Aug 2024 13:06:14 +0100 Subject: [PATCH 011/104] fix tidy --- src/Interpreters/ConcurrentHashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoin.h | 14 ++++++++------ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 772e14a9867..cb069d2fac7 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -208,7 +208,7 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l bool limit_exceeded = !hash_join->data->addBlockToJoin(dispatched_block, check_limits); - dispatched_block = Block{}; + dispatched_block = {}; blocks_left--; if (limit_exceeded) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 96a325ec367..45e6a739a5d 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -143,7 +143,9 @@ public: struct ScatteredBlock : private boost::noncopyable { - ScatteredBlock(const Block & block_) : block(block_), selector(createTrivialSelector(block.rows())) { } + ScatteredBlock() = default; + + explicit ScatteredBlock(const Block & block_) : block(block_), selector(createTrivialSelector(block.rows())) { } ScatteredBlock(const Block & block_, IColumn::Selector && selector_) : block(block_), selector(std::move(selector_)) { } @@ -173,7 +175,7 @@ public: const auto & getSelector() const { return selector; } - operator bool() const { return !!block; } + explicit operator bool() const { return !!block; } /// Accounts only selected rows size_t rows() const { return selector.size(); } @@ -195,7 +197,7 @@ public: void filter(const IColumn::Filter & filter) { chassert(block && block.rows() == filter.size()); - auto it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); + auto * it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); selector.resize(std::distance(selector.begin(), it)); } @@ -215,7 +217,7 @@ public: } /// We have to to id that way because references to the block should remain valid - block.setColumns(std::move(columns)); + block.setColumns(columns); selector = createTrivialSelector(block.rows()); } @@ -225,7 +227,7 @@ public: SCOPE_EXIT(filterBySelector()); if (num_rows >= rows()) - return block.cloneEmpty(); + return ScatteredBlock{block.cloneEmpty()}; chassert(block); @@ -255,7 +257,7 @@ public: } /// We have to to id that way because references to the block should remain valid - block.setColumns(std::move(columns)); + block.setColumns(columns); selector = createTrivialSelector(block.rows()); } From 01179a1eb4cf37227fc85a8e073c20f07f7428a9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Aug 2024 13:53:08 +0100 Subject: [PATCH 012/104] disable check --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 1c08a951c6c..20002225cbc 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -379,7 +379,7 @@ size_t HashJoin::getTotalByteCount() const if (!data) return 0; -#ifndef NDEBUG +#if !defined(NDEBUG) && 0 size_t debug_blocks_allocated_size = 0; for (const auto & block : data->blocks) debug_blocks_allocated_size += block.allocatedBytes(); From 7f69df63fdd80d43d1eb0a9be5d7fc17101b04af Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Aug 2024 15:00:34 +0100 Subject: [PATCH 013/104] small opt --- src/Interpreters/HashJoin/HashJoin.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 45e6a739a5d..a25e6b6fb7c 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -205,6 +206,10 @@ public: void filterBySelector() { chassert(block); + + if (!wasScattered()) + return; + auto columns = block.getColumns(); for (auto & col : columns) { From 6b021c6896d95d86f2076a1e07ac7d0420b168a4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Aug 2024 16:18:17 +0100 Subject: [PATCH 014/104] stash --- src/Interpreters/ConcurrentHashJoin.cpp | 5 ++- src/Interpreters/HashJoin/AddedColumns.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 48 +++++++++++------------- src/Interpreters/HashJoin/HashJoin.h | 7 ++-- 4 files changed, 29 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index cb069d2fac7..07fb6904f97 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -60,7 +60,7 @@ Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) Blocks inner_blocks; for (const auto & block : blocks) { - chassert(!block.wasScattered(), "Not scattered block is expected here"); + chassert(!block.wasScattered(), "Not scattered blocks are expected in join result"); inner_blocks.push_back(block.getSourceBlock()); } return concatenateBlocks(inner_blocks); @@ -178,7 +178,8 @@ ConcurrentHashJoin::~ConcurrentHashJoin() bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_limits) { - /// We prematurely materialize columns here to avoid materializing columns multiple times on each thread. + /// We materialize columns here to avoid materializing them multiple times on different threads + /// (inside different `hash_join`-s) because the block will be shared. Block right_block = hash_joins[0]->data->materializeColumnsFromRightBlock(right_block_); auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index fb6e32efbb8..91db042bdb5 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -34,7 +34,7 @@ struct JoinOnKeyColumns bool isRowFiltered(size_t i) const { - chassert(std::ranges::find(block.getSelector(), i) != block.getSelector().end(), fmt::format("Row {} is not in block", i)); + chassert(block.contains(i), fmt::format("Row {} is not in block", i)); return join_mask_column.isRowFiltered(i); } }; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 20002225cbc..4852be98743 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -81,10 +81,7 @@ Block filterColumnsPresentInSampleBlock(const Block & block, const Block & sampl { Block filtered_block; for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) - { - ColumnWithTypeAndName column = block.getByName(sample_column.name); - filtered_block.insert(std::move(column)); - } + filtered_block.insert(block.getByName(sample_column.name)); return filtered_block; } @@ -456,8 +453,8 @@ Block HashJoin::materializeColumnsFromRightBlock(Block block) const Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block_sample_) { - Block structured_block = DB::materializeColumnsFromRightBlock(block, saved_block_sample_, {}); - return filterColumnsPresentInSampleBlock(structured_block, saved_block_sample_); + Block prepared_block = DB::materializeColumnsFromRightBlock(block, saved_block_sample_, {}); + return filterColumnsPresentInSampleBlock(prepared_block, saved_block_sample_); } Block HashJoin::prepareRightBlock(const Block & block) const @@ -975,6 +972,8 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); + chassert(kind == JoinKind::Left || kind == JoinKind::Inner); + for (const auto & onexpr : table_join->getClauses()) { auto cond_column_name = onexpr.condColumnNames(); @@ -987,31 +986,26 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) cond_column_name.second); } - chassert(kind == JoinKind::Left || kind == JoinKind::Inner); - std::vectormaps[0])> *> maps_vector; for (size_t i = 0; i < table_join->getClauses().size(); ++i) maps_vector.push_back(&data->maps[i]); - if (joinDispatch( - kind, - strictness, - maps_vector, - [&](auto kind_, auto strictness_, auto & maps_vector_) - { - using MapType = typename MapGetter::Map; - ScatteredBlock remaining_block = HashJoinMethods::joinBlockImpl( - *this, block, sample_block_with_columns_to_add, maps_vector_); - if (remaining_block.rows()) - not_processed = std::make_shared(std::move(remaining_block).getSourceBlock()); - else - not_processed.reset(); - })) - { - /// Joined - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); + const bool joined = joinDispatch( + kind, + strictness, + maps_vector, + [&](auto kind_, auto strictness_, auto & maps_vector_) + { + using MapType = typename MapGetter::Map; + ScatteredBlock remaining_block + = HashJoinMethods::joinBlockImpl(*this, block, sample_block_with_columns_to_add, maps_vector_); + if (remaining_block.rows()) + not_processed = std::make_shared(std::move(remaining_block).getSourceBlock()); + else + not_processed.reset(); + }); + + chassert(joined); } HashJoin::~HashJoin() diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index a25e6b6fb7c..2f8a12a4a01 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -146,9 +146,9 @@ public: { ScatteredBlock() = default; - explicit ScatteredBlock(const Block & block_) : block(block_), selector(createTrivialSelector(block.rows())) { } + explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(createTrivialSelector(block.rows())) { } - ScatteredBlock(const Block & block_, IColumn::Selector && selector_) : block(block_), selector(std::move(selector_)) { } + ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) { @@ -176,6 +176,8 @@ public: const auto & getSelector() const { return selector; } + bool contains(size_t idx) const { return std::find(selector.begin(), selector.end(), idx) != selector.end(); } + explicit operator bool() const { return !!block; } /// Accounts only selected rows @@ -261,7 +263,6 @@ public: columns[pos] = std::move(c); } - /// We have to to id that way because references to the block should remain valid block.setColumns(columns); selector = createTrivialSelector(block.rows()); } From e8585a3740641874302d4ec545c2aa58b3935ead Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 17:47:55 +0100 Subject: [PATCH 015/104] stash --- src/Interpreters/ConcurrentHashJoin.cpp | 3 +-- src/Interpreters/HashJoin/HashJoin.cpp | 17 +++++++++++++---- src/Interpreters/HashJoin/HashJoin.h | 1 + src/Interpreters/HashJoin/HashJoinMethods.h | 9 --------- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 07fb6904f97..ccbcb00a6a8 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -225,8 +225,7 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) { - if (hash_joins[0]->data->getKind() == JoinKind::Right || hash_joins[0]->data->getKind() == JoinKind::Full) - materializeBlockInplace(block); + hash_joins[0]->data->materializeColumnsFromLeftBlock(block); auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); block = {}; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 558441fb05a..aa3cc914913 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -446,6 +446,18 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) } } +void HashJoin::materializeColumnsFromLeftBlock(Block & block) const +{ + /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. + * Because if they are constants, then in the "not joined" rows, they may have different values + * - default values, which can differ from the values of these constants. + */ + if (kind == JoinKind::Right || kind == JoinKind::Full) + { + materializeBlockInplace(block); + } +} + Block HashJoin::materializeColumnsFromRightBlock(Block block) const { return DB::materializeColumnsFromRightBlock(std::move(block), savedBlockSample(), table_join->getAllNames(JoinTableSide::Right)); @@ -943,10 +955,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) return; } - if (kind == JoinKind::Right || kind == JoinKind::Full) - { - materializeBlockInplace(block); - } + materializeColumnsFromLeftBlock(block); { std::vectormaps[0])> * > maps_vector; diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 524f9925d83..bc31c4a434e 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -526,6 +526,7 @@ public: void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } + void materializeColumnsFromLeftBlock(Block & block) const; Block materializeColumnsFromRightBlock(Block block) const; private: diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 5acf0c51d3d..7a290087b5e 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -140,15 +140,6 @@ public: auto & source_block = block.getSourceBlock(); size_t existing_columns = source_block.columns(); - /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. - * Because if they are constants, then in the "not joined" rows, they may have different values - * - default values, which can differ from the values of these constants. - */ - if constexpr (join_features.right || join_features.full) - { - materializeBlockInplace(source_block); - } - /** For LEFT/INNER JOIN, the saved blocks do not contain keys. * For FULL/RIGHT JOIN, the saved blocks contain keys; * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. From 5b187b62f0bca6f7f931bcbed1ba95dbec553814 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 18:16:19 +0100 Subject: [PATCH 016/104] fix tidy --- src/Interpreters/ConcurrentHashJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 8e75bcd874b..5c32ebb32f0 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -47,7 +47,7 @@ public: std::string getName() const override { return "ConcurrentHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } - bool addBlockToJoin(const Block & block, bool check_limits) override; + bool addBlockToJoin(const Block & right_block_, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; void joinBlock(Block & block, std::shared_ptr & not_processed) override; void setTotals(const Block & block) override; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index aa3cc914913..1285cdf5035 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -506,7 +506,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) /// We support only INNER/LEFT ASOF join, so rows with NULLs never return from the right joined table. /// So filter them out here not to handle in implementation. const auto & asof_key_name = table_join->getOnlyClause().key_names_right.back(); - auto & asof_column = source_block.getByName(asof_key_name); + const auto & asof_column = source_block.getByName(asof_key_name); if (asof_column.type->isNullable()) { From eb8af558f8e52d296311eff45ec212ccf7f0232b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 20:21:27 +0100 Subject: [PATCH 017/104] stash --- src/Interpreters/HashJoin/HashJoin.h | 163 ++------------------- src/Interpreters/HashJoin/ScatteredBlock.h | 151 +++++++++++++++++++ 2 files changed, 160 insertions(+), 154 deletions(-) create mode 100644 src/Interpreters/HashJoin/ScatteredBlock.h diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index bc31c4a434e..99f2e58069d 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -14,25 +14,19 @@ #include #include -#include -#include -#include -#include -#include - -#include #include - -#include - +#include #include - -#include +#include #include #include - -#include <__ranges/ref_view.h> -#include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -142,145 +136,6 @@ public: return std::make_shared(table_join_, right_sample_block_, any_take_last_row, reserve_num, instance_id); } - struct ScatteredBlock : private boost::noncopyable - { - ScatteredBlock() = default; - - explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(createTrivialSelector(block.rows())) { } - - ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } - - ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) - { - other.block.clear(); - other.selector.clear(); - } - - ScatteredBlock & operator=(ScatteredBlock && other) noexcept - { - if (this != &other) - { - block = std::move(other.block); - selector = std::move(other.selector); - - other.block.clear(); - other.selector.clear(); - } - return *this; - } - - Block & getSourceBlock() & { return block; } - const Block & getSourceBlock() const & { return block; } - - Block && getSourceBlock() && { return std::move(block); } - - const auto & getSelector() const { return selector; } - - bool contains(size_t idx) const { return std::find(selector.begin(), selector.end(), idx) != selector.end(); } - - explicit operator bool() const { return !!block; } - - /// Accounts only selected rows - size_t rows() const { return selector.size(); } - - /// Whether block was scattered, i.e. has non-trivial selector - bool wasScattered() const - { - chassert(block); - return selector.size() != block.rows(); - } - - const ColumnWithTypeAndName & getByName(const std::string & name) const - { - chassert(block); - return block.getByName(name); - } - - /// Filters selector by mask discarding rows for which filter is false - void filter(const IColumn::Filter & filter) - { - chassert(block && block.rows() == filter.size()); - auto * it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); - selector.resize(std::distance(selector.begin(), it)); - } - - /// Applies selector to block in place - void filterBySelector() - { - chassert(block); - - if (!wasScattered()) - return; - - auto columns = block.getColumns(); - for (auto & col : columns) - { - auto c = col->cloneEmpty(); - c->reserve(selector.size()); - /// TODO: create new method in IColumnHelper to devirtualize - for (const auto idx : selector) - c->insertFrom(*col, idx); - col = std::move(c); - } - - /// We have to to id that way because references to the block should remain valid - block.setColumns(columns); - selector = createTrivialSelector(block.rows()); - } - - /// Cut first num_rows rows from block in place and returns block with remaining rows - ScatteredBlock cut(size_t num_rows) - { - SCOPE_EXIT(filterBySelector()); - - if (num_rows >= rows()) - return ScatteredBlock{block.cloneEmpty()}; - - chassert(block); - - IColumn::Selector remaining_selector(selector.begin() + num_rows, selector.end()); - auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; - - selector.erase(selector.begin() + num_rows, selector.end()); - - return remaining; - } - - void replicate(const IColumn::Offsets & offsets, size_t existing_columns, const std::vector & right_keys_to_replicate) - { - chassert(block); - chassert(offsets.size() == rows()); - - auto columns = block.getColumns(); - for (size_t i = 0; i < existing_columns; ++i) - { - auto c = columns[i]->replicate(offsets); - columns[i] = std::move(c); - } - for (size_t pos : right_keys_to_replicate) - { - auto c = columns[pos]->replicate(offsets); - columns[pos] = std::move(c); - } - - block.setColumns(columns); - selector = createTrivialSelector(block.rows()); - } - - private: - IColumn::Selector createTrivialSelector(size_t size) - { - IColumn::Selector res(size); - std::iota(res.begin(), res.end(), 0); - return res; - } - - Block block; - IColumn::Selector selector; - }; - - using ScatteredBlocks = std::vector; - /** Add block of data from right hand of JOIN to the map. * Returns false, if some limit was exceeded and you should not insert more data. */ diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h new file mode 100644 index 00000000000..8dc3c60eb07 --- /dev/null +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -0,0 +1,151 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +struct ScatteredBlock : private boost::noncopyable +{ + ScatteredBlock() = default; + + explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(createTrivialSelector(block.rows())) { } + + ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } + + ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) + { + other.block.clear(); + other.selector.clear(); + } + + ScatteredBlock & operator=(ScatteredBlock && other) noexcept + { + if (this != &other) + { + block = std::move(other.block); + selector = std::move(other.selector); + + other.block.clear(); + other.selector.clear(); + } + return *this; + } + + Block & getSourceBlock() & { return block; } + const Block & getSourceBlock() const & { return block; } + + Block && getSourceBlock() && { return std::move(block); } + + const auto & getSelector() const { return selector; } + + bool contains(size_t idx) const { return std::find(selector.begin(), selector.end(), idx) != selector.end(); } + + explicit operator bool() const { return !!block; } + + /// Accounts only selected rows + size_t rows() const { return selector.size(); } + + /// Whether block was scattered, i.e. has non-trivial selector + bool wasScattered() const + { + chassert(block); + return selector.size() != block.rows(); + } + + const ColumnWithTypeAndName & getByName(const std::string & name) const + { + chassert(block); + return block.getByName(name); + } + + /// Filters selector by mask discarding rows for which filter is false + void filter(const IColumn::Filter & filter) + { + chassert(block && block.rows() == filter.size()); + auto * it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); + selector.resize(std::distance(selector.begin(), it)); + } + + /// Applies selector to block in place + void filterBySelector() + { + chassert(block); + + if (!wasScattered()) + return; + + auto columns = block.getColumns(); + for (auto & col : columns) + { + auto c = col->cloneEmpty(); + c->reserve(selector.size()); + /// TODO: create new method in IColumnHelper to devirtualize + for (const auto idx : selector) + c->insertFrom(*col, idx); + col = std::move(c); + } + + /// We have to to id that way because references to the block should remain valid + block.setColumns(columns); + selector = createTrivialSelector(block.rows()); + } + + /// Cut first num_rows rows from block in place and returns block with remaining rows + ScatteredBlock cut(size_t num_rows) + { + SCOPE_EXIT(filterBySelector()); + + if (num_rows >= rows()) + return ScatteredBlock{block.cloneEmpty()}; + + chassert(block); + + IColumn::Selector remaining_selector(selector.begin() + num_rows, selector.end()); + auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; + + selector.erase(selector.begin() + num_rows, selector.end()); + + return remaining; + } + + void replicate(const IColumn::Offsets & offsets, size_t existing_columns, const std::vector & right_keys_to_replicate) + { + chassert(block); + chassert(offsets.size() == rows()); + + auto columns = block.getColumns(); + for (size_t i = 0; i < existing_columns; ++i) + { + auto c = columns[i]->replicate(offsets); + columns[i] = std::move(c); + } + for (size_t pos : right_keys_to_replicate) + { + auto c = columns[pos]->replicate(offsets); + columns[pos] = std::move(c); + } + + block.setColumns(columns); + selector = createTrivialSelector(block.rows()); + } + +private: + IColumn::Selector createTrivialSelector(size_t size) + { + IColumn::Selector res(size); + std::iota(res.begin(), res.end(), 0); + return res; + } + + Block block; + IColumn::Selector selector; +}; + +using ScatteredBlocks = std::vector; + +} From 662c67dcc0857dff92153a0ec6c81bfd564069ad Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 22:07:29 +0100 Subject: [PATCH 018/104] stash doesn't work --- src/Interpreters/ConcurrentHashJoin.cpp | 6 +- src/Interpreters/ConcurrentHashJoin.h | 2 +- src/Interpreters/HashJoin/AddedColumns.cpp | 2 +- src/Interpreters/HashJoin/AddedColumns.h | 8 +- src/Interpreters/HashJoin/HashJoinMethods.h | 11 +- src/Interpreters/HashJoin/ScatteredBlock.h | 169 ++++++++++++++++++-- 6 files changed, 171 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index ccbcb00a6a8..e393b0b6e50 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -55,7 +55,7 @@ void updateStatistics(const auto & hash_joins, const DB::StatsCollectingParams & DB::getHashTablesStatistics().update(sum_of_sizes, *median_size, params); } -Block concatenateBlocks(const HashJoin::ScatteredBlocks & blocks) +Block concatenateBlocks(const ScatteredBlocks & blocks) { Blocks inner_blocks; for (const auto & block : blocks) @@ -332,7 +332,7 @@ IColumn::Selector selectDispatchBlock(size_t num_shards, const Strings & key_col return hashToSelector(hash, num_shards); } -HashJoin::ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) { size_t num_shards = hash_joins.size(); IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); @@ -344,7 +344,7 @@ HashJoin::ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_ const size_t shard = selector[i]; selectors[shard].push_back(i); } - HashJoin::ScatteredBlocks result; + ScatteredBlocks result; result.reserve(num_shards); for (size_t i = 0; i < num_shards; ++i) result.emplace_back(from_block, std::move(selectors[i])); diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 5c32ebb32f0..b6db783beb1 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -78,7 +78,7 @@ private: std::mutex totals_mutex; Block totals; - HashJoin::ScatteredBlocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); + ScatteredBlocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); }; UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression); diff --git a/src/Interpreters/HashJoin/AddedColumns.cpp b/src/Interpreters/HashJoin/AddedColumns.cpp index 78b8602ce0e..19b59d86466 100644 --- a/src/Interpreters/HashJoin/AddedColumns.cpp +++ b/src/Interpreters/HashJoin/AddedColumns.cpp @@ -4,7 +4,7 @@ namespace DB { JoinOnKeyColumns::JoinOnKeyColumns( - const HashJoin::ScatteredBlock & block_, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) + const ScatteredBlock & block_, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) : block(block_) , key_names(key_names_) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 91db042bdb5..137d5febc46 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -14,7 +14,7 @@ using ExpressionActionsPtr = std::shared_ptr; struct JoinOnKeyColumns { - const HashJoin::ScatteredBlock & block; + const ScatteredBlock & block; Names key_names; @@ -30,7 +30,7 @@ struct JoinOnKeyColumns Sizes key_sizes; JoinOnKeyColumns( - const HashJoin::ScatteredBlock & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_); + const ScatteredBlock & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_); bool isRowFiltered(size_t i) const { @@ -62,7 +62,7 @@ public: }; AddedColumns( - const HashJoin::ScatteredBlock & left_block_, + const ScatteredBlock & left_block_, const Block & block_with_columns_to_add, const Block & saved_block_sample, const HashJoin & join, @@ -142,7 +142,7 @@ public: const IColumn & leftAsofKey() const { return *left_asof_key; } - const HashJoin::ScatteredBlock & src_block; + const ScatteredBlock & src_block; Block left_block; std::vector join_on_keys; ExpressionActionsPtr additional_filter_expression; diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 7a290087b5e..0c4bc02970b 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -11,6 +11,7 @@ #include #include +#include "Interpreters/HashJoin/ScatteredBlock.h" namespace DB @@ -77,7 +78,7 @@ public: const ColumnRawPtrs & key_columns, const Sizes & key_sizes, Block * stored_block, - const IColumn::Selector & selector, + const ScatteredBlock::Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, @@ -113,7 +114,7 @@ public: const MapsTemplateVector & maps_, bool is_join_get = false) { - HashJoin::ScatteredBlock scattered_block{block}; + ScatteredBlock scattered_block{block}; auto ret = joinBlockImpl(join, scattered_block, block_with_columns_to_add, maps_, is_join_get); ret.filterBySelector(); scattered_block.filterBySelector(); @@ -121,9 +122,9 @@ public: return ret.getSourceBlock(); } - static HashJoin::ScatteredBlock joinBlockImpl( + static ScatteredBlock joinBlockImpl( const HashJoin & join, - HashJoin::ScatteredBlock & block, + ScatteredBlock & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get = false) @@ -247,7 +248,7 @@ private: const ColumnRawPtrs & key_columns, const Sizes & key_sizes, Block * stored_block, - const IColumn::Selector & selector, + const ScatteredBlock::Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 8dc3c60eb07..de86ca6659a 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -3,24 +3,161 @@ #include #include #include +#include "base/defines.h" +#include #include +#include +#include + namespace DB { +namespace detail +{ + +class Selector +{ +public: + using Range = std::pair; + + /// [begin, end) + Selector(size_t begin, size_t end) : data(Range{begin, end}) { } + Selector() : Selector(0, 0) { } + + Selector(IColumn::Selector && selector_) : data(initializeFromSelector(std::move(selector_))) { } + + class Iterator + { + public: + using iterator_category = std::forward_iterator_tag; + using value_type = size_t; + using difference_type = std::ptrdiff_t; + using pointer = size_t *; + using reference = size_t &; + + Iterator(const Selector & selector_, size_t idx_) : selector(selector_), idx(idx_) { } + + size_t operator*() const + { + chassert(idx < selector.size()); + if (idx >= selector.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, selector.size()); + return selector[idx]; + } + + Iterator & operator++() + { + if (idx >= selector.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, selector.size()); + ++idx; + return *this; + } + + bool operator!=(const Iterator & other) const { return idx != other.idx; } + + private: + const Selector & selector; + size_t idx; + }; + + Iterator begin() const { return Iterator(*this, 0); } + + Iterator end() const { return Iterator(*this, size()); } + + size_t operator[](size_t idx) const + { + chassert(idx < size()); + if (idx >= size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, size()); + + if (std::holds_alternative(data)) + { + auto range = std::get(data); + return range.first + idx; + } + else + { + return std::get(data)[idx]; + } + } + + size_t size() const + { + if (std::holds_alternative(data)) + { + auto range = std::get(data); + return range.second - range.first; + } + else + { + return std::get(data).size(); + } + } + + std::pair split(size_t num_rows) + { + if (num_rows > size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", num_rows, size()); + + if (std::holds_alternative(data)) + { + auto range = std::get(data); + + if (num_rows == 0) + return {Selector(), Selector{range.first, range.second}}; + + if (num_rows == size()) + return {Selector{range.first, range.second}, Selector()}; + + return {Selector(range.first, range.first + num_rows), Selector(range.first + num_rows, range.second)}; + } + else + { + auto & selector = std::get(data); + return { + Selector(IColumn::Selector(selector.begin(), selector.begin() + num_rows)), + Selector(IColumn::Selector(selector.begin() + num_rows, selector.end()))}; + } + } + +private: + using Data = std::variant; + + Data initializeFromSelector(IColumn::Selector && selector) + { + if (selector.empty()) + return Range{0, 0}; + + /// selector represents continuous range + if (selector.back() == selector.front() + selector.size() - 1) + return Range{selector.front(), selector.front() + selector.size()}; + + return std::move(selector); + } + + Data data; +}; + +} + struct ScatteredBlock : private boost::noncopyable { + using Selector = detail::Selector; + ScatteredBlock() = default; explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(createTrivialSelector(block.rows())) { } ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } + ScatteredBlock(Block block_, Selector selector_) : block(std::move(block_)), selector(std::move(selector_)) { } + ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector)) { other.block.clear(); - other.selector.clear(); + other.selector = {}; } ScatteredBlock & operator=(ScatteredBlock && other) noexcept @@ -31,7 +168,7 @@ struct ScatteredBlock : private boost::noncopyable selector = std::move(other.selector); other.block.clear(); - other.selector.clear(); + other.selector = {}; } return *this; } @@ -67,8 +204,10 @@ struct ScatteredBlock : private boost::noncopyable void filter(const IColumn::Filter & filter) { chassert(block && block.rows() == filter.size()); - auto * it = std::remove_if(selector.begin(), selector.end(), [&](size_t idx) { return !filter[idx]; }); - selector.resize(std::distance(selector.begin(), it)); + IColumn::Selector new_selector; + new_selector.reserve(selector.size()); + std::copy_if(selector.begin(), selector.end(), std::back_inserter(new_selector), [&](size_t idx) { return filter[idx]; }); + selector = std::move(new_selector); } /// Applies selector to block in place @@ -105,10 +244,19 @@ struct ScatteredBlock : private boost::noncopyable chassert(block); - IColumn::Selector remaining_selector(selector.begin() + num_rows, selector.end()); + LOG_DEBUG(&Poco::Logger::get("debug"), "selector=({})", fmt::join(selector, ",")); + + auto && [first_num_rows, remaining_selector] = selector.split(num_rows); + + LOG_DEBUG( + &Poco::Logger::get("debug"), + "first_num_rows=({}), remaining_selector=({})", + fmt::join(first_num_rows, ","), + fmt::join(remaining_selector, ",")); + auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; - selector.erase(selector.begin() + num_rows, selector.end()); + selector = std::move(first_num_rows); return remaining; } @@ -135,15 +283,10 @@ struct ScatteredBlock : private boost::noncopyable } private: - IColumn::Selector createTrivialSelector(size_t size) - { - IColumn::Selector res(size); - std::iota(res.begin(), res.end(), 0); - return res; - } + Selector createTrivialSelector(size_t size) { return Selector(0, size - 1); } Block block; - IColumn::Selector selector; + Selector selector; }; using ScatteredBlocks = std::vector; From 309ee4351f84b17ffafad81d5ae95e98ab688336 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 23:01:46 +0100 Subject: [PATCH 019/104] stash does work --- src/Interpreters/HashJoin/ScatteredBlock.h | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index de86ca6659a..7e606e91a1f 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -244,16 +244,8 @@ struct ScatteredBlock : private boost::noncopyable chassert(block); - LOG_DEBUG(&Poco::Logger::get("debug"), "selector=({})", fmt::join(selector, ",")); - auto && [first_num_rows, remaining_selector] = selector.split(num_rows); - LOG_DEBUG( - &Poco::Logger::get("debug"), - "first_num_rows=({}), remaining_selector=({})", - fmt::join(first_num_rows, ","), - fmt::join(remaining_selector, ",")); - auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; selector = std::move(first_num_rows); @@ -283,7 +275,7 @@ struct ScatteredBlock : private boost::noncopyable } private: - Selector createTrivialSelector(size_t size) { return Selector(0, size - 1); } + Selector createTrivialSelector(size_t size) { return Selector(0, size); } Block block; Selector selector; From 7596d5a0cac1ca06aa6620d9c0857f542339f556 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 11 Aug 2024 19:52:35 +0100 Subject: [PATCH 020/104] fix style --- src/Interpreters/HashJoin/ScatteredBlock.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 7e606e91a1f..5e67075b91b 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + namespace detail { From 101eeae888e7db22b81096173f2ecd91bf9b2807 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 19:23:02 +0100 Subject: [PATCH 021/104] optimize --- src/Interpreters/HashJoin/ScatteredBlock.h | 43 +++++++++++++++++++--- 1 file changed, 38 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 5e67075b91b..3668750f044 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -29,6 +29,7 @@ public: /// [begin, end) Selector(size_t begin, size_t end) : data(Range{begin, end}) { } + Selector(size_t size) : Selector(0, size) { } Selector() : Selector(0, 0) { } Selector(IColumn::Selector && selector_) : data(initializeFromSelector(std::move(selector_))) { } @@ -127,6 +128,28 @@ public: } } + bool isContinuousRange() const { return std::holds_alternative(data); } + + Range getRange() const + { + chassert(isContinuousRange()); + return std::get(data); + } + + std::string toString() const + { + if (std::holds_alternative(data)) + { + auto range = std::get(data); + return fmt::format("[{}, {})", range.first, range.second); + } + else + { + auto & selector = std::get(data); + return fmt::format("({})", fmt::join(selector, ",")); + } + } + private: using Data = std::variant; @@ -153,7 +176,7 @@ struct ScatteredBlock : private boost::noncopyable ScatteredBlock() = default; - explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(createTrivialSelector(block.rows())) { } + explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(block.rows()) { } ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } @@ -223,6 +246,18 @@ struct ScatteredBlock : private boost::noncopyable if (!wasScattered()) return; + if (selector.isContinuousRange()) + { + const auto range = selector.getRange(); + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i); + col.column = col.column->cut(range.first, range.second - range.first); + } + selector = Selector(block.rows()); + return; + } + auto columns = block.getColumns(); for (auto & col : columns) { @@ -236,7 +271,7 @@ struct ScatteredBlock : private boost::noncopyable /// We have to to id that way because references to the block should remain valid block.setColumns(columns); - selector = createTrivialSelector(block.rows()); + selector = Selector(block.rows()); } /// Cut first num_rows rows from block in place and returns block with remaining rows @@ -276,12 +311,10 @@ struct ScatteredBlock : private boost::noncopyable } block.setColumns(columns); - selector = createTrivialSelector(block.rows()); + selector = Selector(block.rows()); } private: - Selector createTrivialSelector(size_t size) { return Selector(0, size); } - Block block; Selector selector; }; From 532bda833444fb81dcea1a34d52a446dc3a1fbaf Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 13:42:35 +0100 Subject: [PATCH 022/104] optimize --- src/Interpreters/HashJoin/ScatteredBlock.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 3668750f044..4952ce1808c 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -45,7 +45,7 @@ public: Iterator(const Selector & selector_, size_t idx_) : selector(selector_), idx(idx_) { } - size_t operator*() const + size_t ALWAYS_INLINE operator*() const { chassert(idx < selector.size()); if (idx >= selector.size()) @@ -53,7 +53,7 @@ public: return selector[idx]; } - Iterator & operator++() + Iterator & ALWAYS_INLINE operator++() { if (idx >= selector.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, selector.size()); @@ -61,7 +61,7 @@ public: return *this; } - bool operator!=(const Iterator & other) const { return idx != other.idx; } + bool ALWAYS_INLINE operator!=(const Iterator & other) const { return idx != other.idx; } private: const Selector & selector; @@ -72,7 +72,7 @@ public: Iterator end() const { return Iterator(*this, size()); } - size_t operator[](size_t idx) const + size_t ALWAYS_INLINE operator[](size_t idx) const { chassert(idx < size()); if (idx >= size()) From 1c6eafbfcfeb75d6e01d8749687f1a03fc181d65 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 14:21:25 +0100 Subject: [PATCH 023/104] stash --- src/Interpreters/HashJoin/ScatteredBlock.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 4952ce1808c..96ec29759e6 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -75,8 +75,6 @@ public: size_t ALWAYS_INLINE operator[](size_t idx) const { chassert(idx < size()); - if (idx >= size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, size()); if (std::holds_alternative(data)) { @@ -104,8 +102,7 @@ public: std::pair split(size_t num_rows) { - if (num_rows > size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", num_rows, size()); + chassert(num_rows <= size()); if (std::holds_alternative(data)) { From e3caa59f839e345427359f4eae6661c0342fd6c3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 21:09:17 +0100 Subject: [PATCH 024/104] too expensive --- src/Interpreters/HashJoin/AddedColumns.h | 1 - src/Interpreters/HashJoin/ScatteredBlock.h | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 137d5febc46..08b5f71f222 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -34,7 +34,6 @@ struct JoinOnKeyColumns bool isRowFiltered(size_t i) const { - chassert(block.contains(i), fmt::format("Row {} is not in block", i)); return join_mask_column.isRowFiltered(i); } }; diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 96ec29759e6..bde5796ae4c 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -205,8 +205,6 @@ struct ScatteredBlock : private boost::noncopyable const auto & getSelector() const { return selector; } - bool contains(size_t idx) const { return std::find(selector.begin(), selector.end(), idx) != selector.end(); } - explicit operator bool() const { return !!block; } /// Accounts only selected rows From 9c55ad799bd92d95c627a10ea895c20b73c47eca Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 14 Aug 2024 16:41:37 +0100 Subject: [PATCH 025/104] stash half --- src/Interpreters/HashJoin/HashJoinMethods.h | 22 +++++++++++++++ .../HashJoin/HashJoinMethodsImpl.h | 28 ++++++++++++++++++- src/Interpreters/HashJoin/ScatteredBlock.h | 6 ++++ 3 files changed, 55 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 298671b8271..6e741628234 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -108,6 +108,20 @@ private: Arena & pool, bool & is_inserted); + template + static size_t insertFromBlockImplTypeCase( + HashJoin & join, + HashMap & map, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + const Selector & selector, + size_t rows, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted); + template static size_t switchJoinRightColumns( const std::vector & mapv, @@ -138,6 +152,14 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags); + template + static size_t joinRightColumns( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags, + const Selector & selector); + template static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index e3f57e5bf64..5c328c66f95 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -1,5 +1,8 @@ #pragma once +#include #include +#include "Columns/IColumn.h" +#include "Interpreters/HashJoin/ScatteredBlock.h" namespace DB { @@ -334,6 +337,24 @@ size_t HashJoinMethods::joinRightColumns( const std::vector & mapv, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags) +{ + auto & block = added_columns.src_block; + if (block.getSelector().isContinuousRange()) + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange()); + else + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); +} + +template +template +size_t HashJoinMethods::joinRightColumns( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags, + const Selector & selector) { constexpr JoinFeatures join_features; @@ -352,7 +373,12 @@ size_t HashJoinMethods::joinRightColumns( size_t i = 0; for (; i < rows; ++i) { - const auto ind = block.getSelector()[i]; + size_t ind = 0; + if constexpr (std::is_same_v, IColumn::Selector>) + ind = selector[i]; + else + ind = selector.first + i; + if constexpr (join_features.need_replication) { if (unlikely(current_offset >= max_joined_block_rows)) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index bde5796ae4c..f28126212d7 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -133,6 +133,12 @@ public: return std::get(data); } + const IColumn::Selector & getSelector() const + { + chassert(!isContinuousRange()); + return std::get(data); + } + std::string toString() const { if (std::holds_alternative(data)) From 4f3ff0ecbd860e62cd8026811c06dcf64d1c25d8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 14 Aug 2024 19:31:54 +0100 Subject: [PATCH 026/104] second half --- src/Interpreters/HashJoin/HashJoinMethods.h | 21 ------ .../HashJoin/HashJoinMethodsImpl.h | 69 +++++++++++-------- 2 files changed, 42 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 6e741628234..c44a03aa15e 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -95,19 +95,6 @@ private: template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); - template - static size_t insertFromBlockImplTypeCase( - HashJoin & join, - HashMap & map, - const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, - Block * stored_block, - const ScatteredBlock::Selector & selector, - ConstNullMapPtr null_map, - UInt8ColumnDataPtr join_mask, - Arena & pool, - bool & is_inserted); - template static size_t insertFromBlockImplTypeCase( HashJoin & join, @@ -116,7 +103,6 @@ private: const Sizes & key_sizes, Block * stored_block, const Selector & selector, - size_t rows, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, @@ -145,13 +131,6 @@ private: /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). - template - static size_t joinRightColumns( - std::vector && key_getter_vector, - const std::vector & mapv, - AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags); - template static size_t joinRightColumns( std::vector && key_getter_vector, diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 5c328c66f95..6292bce5425 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -36,9 +36,14 @@ size_t HashJoinMethods::insertFromBlockImpl( #define M(TYPE) \ case HashJoin::Type::TYPE: \ - return insertFromBlockImplTypeCase< \ - typename KeyGetterForType>::Type>( \ - join, *maps.TYPE, key_columns, key_sizes, stored_block, selector, null_map, join_mask, pool, is_inserted); \ + if (selector.isContinuousRange()) \ + return insertFromBlockImplTypeCase< \ + typename KeyGetterForType>::Type>( \ + join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getRange(), null_map, join_mask, pool, is_inserted); \ + else \ + return insertFromBlockImplTypeCase< \ + typename KeyGetterForType>::Type>( \ + join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getSelector(), null_map, join_mask, pool, is_inserted); \ break; APPLY_FOR_JOIN_VARIANTS(M) @@ -179,14 +184,14 @@ KeyGetter HashJoinMethods::createKeyGetter(const } template -template +template size_t HashJoinMethods::insertFromBlockImplTypeCase( HashJoin & join, HashMap & map, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, Block * stored_block, - const ScatteredBlock::Selector & selector, + const Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, @@ -204,8 +209,20 @@ size_t HashJoinMethods::insertFromBlockImplTypeC /// For ALL and ASOF join always insert values is_inserted = !mapped_one || is_asof_join; - for (size_t ind : selector) + size_t rows = 0; + if constexpr (std::is_same_v, IColumn::Selector>) + rows = selector.size(); + else + rows = selector.second - selector.first; + + for (size_t i = 0; i < rows; ++i) { + size_t ind = 0; + if constexpr (std::is_same_v, IColumn::Selector>) + ind = selector[i]; + else + ind = selector.first + i; + chassert(!null_map || ind < null_map->size()); if (null_map && (*null_map)[ind]) { @@ -321,32 +338,30 @@ size_t HashJoinMethods::joinRightColumnsSwitchMu if (added_columns.additional_filter_expression) throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); - return mapv.size() > 1 ? joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); + auto & block = added_columns.src_block; + if (block.getSelector().isContinuousRange()) + { + if (mapv.size() > 1) + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange()); + else + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange()); + } + else + { + if (mapv.size() > 1) + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); + else + return joinRightColumns( + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); + } } /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template -template -size_t HashJoinMethods::joinRightColumns( - std::vector && key_getter_vector, - const std::vector & mapv, - AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) -{ - auto & block = added_columns.src_block; - if (block.getSelector().isContinuousRange()) - return joinRightColumns( - std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange()); - else - return joinRightColumns( - std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); -} - template template size_t HashJoinMethods::joinRightColumns( From 3f5f59242e3b93a673fa91b9cdbab26b7c5e1cbd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 15 Aug 2024 20:06:14 +0100 Subject: [PATCH 027/104] fix build --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index a222fed0a6e..a6c11f60cf4 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -429,16 +429,8 @@ size_t HashJoinMethods::joinRightColumns( { const IColumn & left_asof_key = added_columns.leftAsofKey(); -<<<<<<< HEAD auto row_ref = mapped->findAsof(left_asof_key, ind); - if (row_ref.block) -||||||| 014c1650f8d - auto row_ref = mapped->findAsof(left_asof_key, i); - if (row_ref.block) -======= - auto row_ref = mapped->findAsof(left_asof_key, i); if (row_ref && row_ref->block) ->>>>>>> master { setUsed(added_columns.filter, i); if constexpr (flag_per_row) From 95928475873204e55d2cef950e6abb2363806178 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 13:38:02 +0100 Subject: [PATCH 028/104] use index() --- src/Columns/ColumnVector.h | 1 + src/Interpreters/ConcurrentHashJoin.cpp | 10 ++-- .../HashJoin/HashJoinMethodsImpl.h | 18 +++---- src/Interpreters/HashJoin/ScatteredBlock.h | 54 +++++++++---------- 4 files changed, 44 insertions(+), 39 deletions(-) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index a5e1ee4b462..6f998f804ec 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -52,6 +52,7 @@ private: explicit ColumnVector(const size_t n) : data(n) {} ColumnVector(const size_t n, const ValueType x) : data(n, x) {} ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {} + ColumnVector(Container::iterator begin, Container::iterator end) : data(begin, end) { } /// Sugar constructor. ColumnVector(std::initializer_list il) : data{il} {} diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index e393b0b6e50..43f0eb9aa85 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -23,6 +23,7 @@ #include #include #include +#include "Interpreters/HashJoin/ScatteredBlock.h" using namespace DB; @@ -336,13 +337,16 @@ ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_na { size_t num_shards = hash_joins.size(); IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); - std::vector selectors(num_shards); + std::vector selectors(num_shards); for (size_t i = 0; i < num_shards; ++i) - selectors[i].reserve(selector.size() / num_shards + 1); + { + selectors[i] = ScatteredBlock::Indexes::create(); + selectors[i]->reserve(selector.size() / num_shards + 1); + } for (size_t i = 0; i < selector.size(); ++i) { const size_t shard = selector[i]; - selectors[shard].push_back(i); + selectors[shard]->getData().push_back(i); } ScatteredBlocks result; result.reserve(num_shards); diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index a6c11f60cf4..47e1fa49697 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -43,7 +43,7 @@ size_t HashJoinMethods::insertFromBlockImpl( else \ return insertFromBlockImplTypeCase< \ typename KeyGetterForType>::Type>( \ - join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getSelector(), null_map, join_mask, pool, is_inserted); \ + join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getIndexes(), null_map, join_mask, pool, is_inserted); \ break; APPLY_FOR_JOIN_VARIANTS(M) @@ -213,16 +213,16 @@ size_t HashJoinMethods::insertFromBlockImplTypeC is_inserted = !mapped_one || is_asof_join; size_t rows = 0; - if constexpr (std::is_same_v, IColumn::Selector>) - rows = selector.size(); + if constexpr (std::is_same_v, ScatteredBlock::Indexes>) + rows = selector.getData().size(); else rows = selector.second - selector.first; for (size_t i = 0; i < rows; ++i) { size_t ind = 0; - if constexpr (std::is_same_v, IColumn::Selector>) - ind = selector[i]; + if constexpr (std::is_same_v, ScatteredBlock::Indexes>) + ind = selector.getData()[i]; else ind = selector.first + i; @@ -355,10 +355,10 @@ size_t HashJoinMethods::joinRightColumnsSwitchMu { if (mapv.size() > 1) return joinRightColumns( - std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getIndexes()); else return joinRightColumns( - std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getSelector()); + std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getIndexes()); } } @@ -394,8 +394,8 @@ size_t HashJoinMethods::joinRightColumns( for (; i < rows; ++i) { size_t ind = 0; - if constexpr (std::is_same_v, IColumn::Selector>) - ind = selector[i]; + if constexpr (std::is_same_v, ScatteredBlock::Indexes>) + ind = selector.getData()[i]; else ind = selector.first + i; diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index f28126212d7..69456898394 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -1,9 +1,11 @@ #pragma once +#include +#include #include #include +#include #include -#include "base/defines.h" #include #include @@ -26,13 +28,15 @@ class Selector { public: using Range = std::pair; + using Indexes = ColumnUInt64; + using IndexesPtr = ColumnUInt64::MutablePtr; /// [begin, end) Selector(size_t begin, size_t end) : data(Range{begin, end}) { } Selector(size_t size) : Selector(0, size) { } Selector() : Selector(0, 0) { } - Selector(IColumn::Selector && selector_) : data(initializeFromSelector(std::move(selector_))) { } + Selector(IndexesPtr && selector_) : data(initializeFromSelector(std::move(selector_))) { } class Iterator { @@ -83,7 +87,7 @@ public: } else { - return std::get(data)[idx]; + return std::get(data)->getData()[idx]; } } @@ -96,7 +100,7 @@ public: } else { - return std::get(data).size(); + return std::get(data)->size(); } } @@ -118,10 +122,10 @@ public: } else { - auto & selector = std::get(data); + auto & selector = std::get(data)->getData(); return { - Selector(IColumn::Selector(selector.begin(), selector.begin() + num_rows)), - Selector(IColumn::Selector(selector.begin() + num_rows, selector.end()))}; + Selector(Indexes::create(selector.begin(), selector.begin() + num_rows)), + Selector(Indexes::create(selector.begin() + num_rows, selector.end()))}; } } @@ -133,10 +137,10 @@ public: return std::get(data); } - const IColumn::Selector & getSelector() const + const Indexes & getIndexes() const { chassert(!isContinuousRange()); - return std::get(data); + return *std::get(data); } std::string toString() const @@ -148,16 +152,17 @@ public: } else { - auto & selector = std::get(data); + auto & selector = std::get(data)->getData(); return fmt::format("({})", fmt::join(selector, ",")); } } private: - using Data = std::variant; + using Data = std::variant; - Data initializeFromSelector(IColumn::Selector && selector) + Data initializeFromSelector(IndexesPtr && selector_) { + const auto & selector = selector_->getData(); if (selector.empty()) return Range{0, 0}; @@ -165,7 +170,7 @@ private: if (selector.back() == selector.front() + selector.size() - 1) return Range{selector.front(), selector.front() + selector.size()}; - return std::move(selector); + return std::move(selector_); } Data data; @@ -176,12 +181,14 @@ private: struct ScatteredBlock : private boost::noncopyable { using Selector = detail::Selector; + using Indexes = Selector::Indexes; + using IndexesPtr = Selector::IndexesPtr; ScatteredBlock() = default; explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(block.rows()) { } - ScatteredBlock(Block block_, IColumn::Selector && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } + ScatteredBlock(Block block_, IndexesPtr && selector_) : block(std::move(block_)), selector(std::move(selector_)) { } ScatteredBlock(Block block_, Selector selector_) : block(std::move(block_)), selector(std::move(selector_)) { } @@ -233,9 +240,10 @@ struct ScatteredBlock : private boost::noncopyable void filter(const IColumn::Filter & filter) { chassert(block && block.rows() == filter.size()); - IColumn::Selector new_selector; - new_selector.reserve(selector.size()); - std::copy_if(selector.begin(), selector.end(), std::back_inserter(new_selector), [&](size_t idx) { return filter[idx]; }); + IndexesPtr new_selector = Indexes::create(); + new_selector->reserve(selector.size()); + std::copy_if( + selector.begin(), selector.end(), std::back_inserter(new_selector->getData()), [&](size_t idx) { return filter[idx]; }); selector = std::move(new_selector); } @@ -259,18 +267,10 @@ struct ScatteredBlock : private boost::noncopyable return; } + /// The general case when selector is non-trivial (likely the result of applying a filter) auto columns = block.getColumns(); for (auto & col : columns) - { - auto c = col->cloneEmpty(); - c->reserve(selector.size()); - /// TODO: create new method in IColumnHelper to devirtualize - for (const auto idx : selector) - c->insertFrom(*col, idx); - col = std::move(c); - } - - /// We have to to id that way because references to the block should remain valid + col = col->index(selector.getIndexes(), /*limit*/ 0); block.setColumns(columns); selector = Selector(block.rows()); } From 42f67904d64f860c18c2e1189ed75cdc046b4185 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 21:05:19 +0100 Subject: [PATCH 029/104] better --- src/Columns/ColumnVector.h | 2 +- src/Interpreters/ConcurrentHashJoin.cpp | 2 +- src/Interpreters/HashJoin/ScatteredBlock.h | 43 ++++++++++------------ 3 files changed, 21 insertions(+), 26 deletions(-) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 6f998f804ec..415a83ad203 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -52,7 +52,7 @@ private: explicit ColumnVector(const size_t n) : data(n) {} ColumnVector(const size_t n, const ValueType x) : data(n, x) {} ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {} - ColumnVector(Container::iterator begin, Container::iterator end) : data(begin, end) { } + ColumnVector(Container::const_iterator begin, Container::const_iterator end) : data(begin, end) { } /// Sugar constructor. ColumnVector(std::initializer_list il) : data{il} {} diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 43f0eb9aa85..77b3bec34dd 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -23,7 +24,6 @@ #include #include #include -#include "Interpreters/HashJoin/ScatteredBlock.h" using namespace DB; diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 69456898394..79d7e8bb7f0 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -31,12 +31,13 @@ public: using Indexes = ColumnUInt64; using IndexesPtr = ColumnUInt64::MutablePtr; - /// [begin, end) - Selector(size_t begin, size_t end) : data(Range{begin, end}) { } - Selector(size_t size) : Selector(0, size) { } Selector() : Selector(0, 0) { } - Selector(IndexesPtr && selector_) : data(initializeFromSelector(std::move(selector_))) { } + /// [begin, end) + Selector(size_t begin, size_t end) : data(Range{begin, end}) { } + + explicit Selector(size_t size) : Selector(0, size) { } + explicit Selector(IndexesPtr && selector_) : data(initializeFromSelector(std::move(selector_))) { } class Iterator { @@ -82,7 +83,7 @@ public: if (std::holds_alternative(data)) { - auto range = std::get(data); + const auto range = std::get(data); return range.first + idx; } else @@ -95,7 +96,7 @@ public: { if (std::holds_alternative(data)) { - auto range = std::get(data); + const auto range = std::get(data); return range.second - range.first; } else @@ -110,7 +111,7 @@ public: if (std::holds_alternative(data)) { - auto range = std::get(data); + const auto range = std::get(data); if (num_rows == 0) return {Selector(), Selector{range.first, range.second}}; @@ -122,10 +123,10 @@ public: } else { - auto & selector = std::get(data)->getData(); - return { - Selector(Indexes::create(selector.begin(), selector.begin() + num_rows)), - Selector(Indexes::create(selector.begin() + num_rows, selector.end()))}; + const auto & selector = std::get(data)->getData(); + auto && left = Selector(Indexes::create(selector.begin(), selector.begin() + num_rows)); + auto && right = Selector(Indexes::create(selector.begin() + num_rows, selector.end())); + return {std::move(left), std::move(right)}; } } @@ -147,12 +148,12 @@ public: { if (std::holds_alternative(data)) { - auto range = std::get(data); + const auto range = std::get(data); return fmt::format("[{}, {})", range.first, range.second); } else { - auto & selector = std::get(data)->getData(); + const auto & selector = std::get(data)->getData(); return fmt::format("({})", fmt::join(selector, ",")); } } @@ -244,7 +245,7 @@ struct ScatteredBlock : private boost::noncopyable new_selector->reserve(selector.size()); std::copy_if( selector.begin(), selector.end(), std::back_inserter(new_selector->getData()), [&](size_t idx) { return filter[idx]; }); - selector = std::move(new_selector); + selector = Selector(std::move(new_selector)); } /// Applies selector to block in place @@ -299,19 +300,13 @@ struct ScatteredBlock : private boost::noncopyable chassert(block); chassert(offsets.size() == rows()); - auto columns = block.getColumns(); + auto && columns = block.getColumns(); for (size_t i = 0; i < existing_columns; ++i) - { - auto c = columns[i]->replicate(offsets); - columns[i] = std::move(c); - } + columns[i] = columns[i]->replicate(offsets); for (size_t pos : right_keys_to_replicate) - { - auto c = columns[pos]->replicate(offsets); - columns[pos] = std::move(c); - } + columns[pos] = columns[pos]->replicate(offsets); - block.setColumns(columns); + block.setColumns(std::move(columns)); selector = Selector(block.rows()); } From ae6ff4cb6855c2d9b500ef92ff250057e94407b4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 16 Aug 2024 13:11:12 +0100 Subject: [PATCH 030/104] stash --- src/Interpreters/HashJoin/ScatteredBlock.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 79d7e8bb7f0..6c2c84e0893 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -282,7 +282,7 @@ struct ScatteredBlock : private boost::noncopyable SCOPE_EXIT(filterBySelector()); if (num_rows >= rows()) - return ScatteredBlock{block.cloneEmpty()}; + return ScatteredBlock{Block{}}; chassert(block); From 97c86b3237fc09456db93079633f2f07eb7fa2c0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 20 Aug 2024 14:14:15 +0100 Subject: [PATCH 031/104] fix assert --- src/Interpreters/HashJoin/ScatteredBlock.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 6c2c84e0893..9847e8c0b7b 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -251,9 +251,7 @@ struct ScatteredBlock : private boost::noncopyable /// Applies selector to block in place void filterBySelector() { - chassert(block); - - if (!wasScattered()) + if (!block || !wasScattered()) return; if (selector.isContinuousRange()) From f62c7012d70f373b0767ff9123f817ef15c61f15 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 20 Aug 2024 16:40:35 +0100 Subject: [PATCH 032/104] fix tidy --- src/Interpreters/HashJoin/ScatteredBlock.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 9847e8c0b7b..2fc115f8345 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -304,7 +304,7 @@ struct ScatteredBlock : private boost::noncopyable for (size_t pos : right_keys_to_replicate) columns[pos] = columns[pos]->replicate(offsets); - block.setColumns(std::move(columns)); + block.setColumns(columns); selector = Selector(block.rows()); } From 16ddf20c73aac6060e63ce7049e4d2f8d49d4915 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 2 Oct 2024 17:29:30 +0100 Subject: [PATCH 033/104] avoid squashing result blocks --- src/Interpreters/ConcurrentHashJoin.cpp | 25 +++++---- src/Interpreters/ConcurrentHashJoin.h | 3 + src/Interpreters/HashJoin/HashJoin.h | 2 + src/Interpreters/IJoin.h | 9 +++ .../Transforms/JoiningTransform.cpp | 55 ++++++++++++------- src/Processors/Transforms/JoiningTransform.h | 9 ++- 6 files changed, 67 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index a1761911a6c..4b88ef14196 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -56,17 +56,6 @@ void updateStatistics(const auto & hash_joins, const DB::StatsCollectingParams & DB::getHashTablesStatistics().update(sum_of_sizes, *median_size, params); } -Block concatenateBlocks(const ScatteredBlocks & blocks) -{ - Blocks inner_blocks; - for (const auto & block : blocks) - { - chassert(!block.wasScattered(), "Not scattered blocks are expected in join result"); - inner_blocks.push_back(block.getSourceBlock()); - } - return concatenateBlocks(inner_blocks); -} - } namespace DB @@ -227,6 +216,14 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l } void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) +{ + Blocks res; + std::shared_ptr not_processed; + joinBlock(block, res, not_processed); + block = concatenateBlocks(res); +} + +void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std::shared_ptr & /*not_processed*/) { hash_joins[0]->data->materializeColumnsFromLeftBlock(block); @@ -242,7 +239,11 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); } - block = ::concatenateBlocks(dispatched_blocks); + chassert(res.empty()); + res.clear(); + res.reserve(dispatched_blocks.size()); + std::ranges::transform( + dispatched_blocks, std::back_inserter(res), [](ScatteredBlock & res_block) { return std::move(res_block).getSourceBlock(); }); } void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index b6db783beb1..33407045c44 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -57,6 +57,9 @@ public: bool alwaysReturnsEmptySet() const override; bool supportParallelJoin() const override { return true; } + bool supportsJoinWithManyResultBlocks() const override { return true; } + void joinBlock(Block & block, std::vector & res, std::shared_ptr & not_processed) override; + IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 7915788382d..a76c60aab5a 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -146,6 +146,8 @@ public: void checkTypesOfKeys(const Block & block) const override; + using IJoin::joinBlock; + /** Join data from the map (that was previously built by calls to addBlockToJoin) to the block with data from "left" table. * Could be called from different threads in parallel. */ diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 8f648de2538..4bfd43598ec 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -90,6 +90,15 @@ public: /// Could be called from different threads in parallel. virtual void joinBlock(Block & block, std::shared_ptr & not_processed) = 0; + virtual bool supportsJoinWithManyResultBlocks() const { return false; } + virtual void joinBlock( + [[maybe_unused]] Block & block, + [[maybe_unused]] std::vector & res, + [[maybe_unused]] std::shared_ptr & not_processed) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Clone method is not supported for {}", getName()); + } + /** Set/Get totals for right table * Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f2fb6327129..ceb95cd7ad1 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -75,8 +75,9 @@ IProcessor::Status JoiningTransform::prepare() /// Output if has data. if (has_output) { - output.push(std::move(output_chunk)); - has_output = false; + output.push(std::move(output_chunks.front())); + output_chunks.pop_front(); + has_output = !output_chunks.empty(); return Status::PortFull; } @@ -122,10 +123,10 @@ void JoiningTransform::work() { if (has_input) { + chassert(output_chunks.empty()); transform(input_chunk); - output_chunk.swap(input_chunk); has_input = not_processed != nullptr; - has_output = !output_chunk.empty(); + has_output = !output_chunks.empty(); } else { @@ -153,8 +154,7 @@ void JoiningTransform::work() return; } - auto rows = block.rows(); - output_chunk.setColumns(block.getColumns(), rows); + output_chunks.emplace_back(block.getColumns(), block.rows()); has_output = true; } } @@ -173,7 +173,7 @@ void JoiningTransform::transform(Chunk & chunk) } } - Block block; + Blocks res; if (on_totals) { const auto & left_totals = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); @@ -184,39 +184,52 @@ void JoiningTransform::transform(Chunk & chunk) if (default_totals && !right_totals) return; - block = outputs.front().getHeader().cloneEmpty(); - JoinCommon::joinTotals(left_totals, right_totals, join->getTableJoin(), block); + res.emplace_back(); + res.back() = outputs.front().getHeader().cloneEmpty(); + JoinCommon::joinTotals(left_totals, right_totals, join->getTableJoin(), res.back()); } else - block = readExecute(chunk); - auto num_rows = block.rows(); - chunk.setColumns(block.getColumns(), num_rows); + res = readExecute(chunk); + + std::ranges::for_each(res, [this](Block & block) { output_chunks.emplace_back(block.getColumns(), block.rows()); }); } -Block JoiningTransform::readExecute(Chunk & chunk) +Blocks JoiningTransform::readExecute(Chunk & chunk) { - Block res; + Blocks res; + Block block; + + auto join_block = [&]() + { + if (join->supportsJoinWithManyResultBlocks()) + join->joinBlock(block, res, not_processed); + else + { + join->joinBlock(block, not_processed); + res.push_back(std::move(block)); + } + }; if (!not_processed) { if (chunk.hasColumns()) - res = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); + block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); - if (res) - join->joinBlock(res, not_processed); + if (block) + join_block(); } else if (not_processed->empty()) /// There's not processed data inside expression. { if (chunk.hasColumns()) - res = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); + block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); not_processed.reset(); - join->joinBlock(res, not_processed); + join_block(); } else { - res = std::move(not_processed->block); - join->joinBlock(res, not_processed); + block = std::move(not_processed->block); + join_block(); } return res; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 5f6d9d6fff2..c3445cf3e36 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -1,6 +1,9 @@ #pragma once -#include + #include +#include + +#include #include namespace DB @@ -66,7 +69,7 @@ protected: private: Chunk input_chunk; - Chunk output_chunk; + std::deque output_chunks; bool has_input = false; bool has_output = false; bool stop_reading = false; @@ -86,7 +89,7 @@ private: IBlocksStreamPtr non_joined_blocks; size_t max_block_size; - Block readExecute(Chunk & chunk); + Blocks readExecute(Chunk & chunk); }; /// Fills Join with block from right table. From ceeb522b3713041d6b6ab00cedf218e1ef9fe245 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 3 Oct 2024 17:35:09 +0100 Subject: [PATCH 034/104] use old approach for thin blocks --- src/Interpreters/ConcurrentHashJoin.cpp | 53 +++++++++++++++++++++++-- 1 file changed, 50 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 4b88ef14196..f2d1d1418a4 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -25,6 +27,8 @@ #include #include +#include + using namespace DB; namespace ProfileEvents @@ -336,10 +340,31 @@ IColumn::Selector selectDispatchBlock(size_t num_shards, const Strings & key_col return hashToSelector(hash, num_shards); } -ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +ScatteredBlocks scatterBlocksByCopying(size_t num_shards, const IColumn::Selector & selector, const Block & from_block) +{ + Blocks blocks(num_shards); + for (size_t i = 0; i < num_shards; ++i) + blocks[i] = from_block.cloneEmpty(); + + for (size_t i = 0; i < from_block.columns(); ++i) + { + auto dispatched_columns = from_block.getByPosition(i).column->scatter(num_shards, selector); + chassert(blocks.size() == dispatched_columns.size()); + for (size_t block_index = 0; block_index < num_shards; ++block_index) + { + blocks[block_index].getByPosition(i).column = std::move(dispatched_columns[block_index]); + } + } + + ScatteredBlocks result; + result.reserve(num_shards); + for (size_t i = 0; i < num_shards; ++i) + result.emplace_back(std::move(blocks[i])); + return result; +} + +ScatteredBlocks scatterBlocksWithSelector(size_t num_shards, const IColumn::Selector & selector, const Block & from_block) { - size_t num_shards = hash_joins.size(); - IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); std::vector selectors(num_shards); for (size_t i = 0; i < num_shards; ++i) { @@ -358,6 +383,28 @@ ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_na return result; } +ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +{ + size_t num_shards = hash_joins.size(); + IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); + + /// With zero-copy approach we won't copy the source columns, but will create a new one with indices. + /// This is not beneficial when the whole set of columns is e.g. a single small column. + constexpr auto threshold = sizeof(IColumn::Selector::value_type); + const auto & data_types = from_block.getDataTypes(); + const bool use_zero_copy_approach + = std::accumulate( + data_types.begin(), + data_types.end(), + 0u, + [](size_t sum, const DataTypePtr & type) + { return sum + (type->haveMaximumSizeOfValue() ? type->getMaximumSizeOfValueInMemory() : threshold + 1); }) + > threshold; + + return use_zero_copy_approach ? scatterBlocksWithSelector(num_shards, selector, from_block) + : scatterBlocksByCopying(num_shards, selector, from_block); +} + UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression) { IQueryTreeNode::HashState hash; From 5cf92fe96404909307759d35fa5f412cd1d5a717 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 14 Oct 2024 14:22:17 +0100 Subject: [PATCH 035/104] better --- .../HashJoin/HashJoinMethodsImpl.h | 15 ++++++- src/Interpreters/HashJoin/ScatteredBlock.h | 40 ++++++------------- 2 files changed, 26 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 350b4c7cfc6..17c8c821bef 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -164,8 +164,19 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( if constexpr (join_features.need_replication) { - std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; - block.replicate(*offsets_to_replicate, existing_columns, right_keys_to_replicate); + IColumn::Offsets & offsets = *added_columns.offsets_to_replicate; + + chassert(block); + chassert(offsets.size() == block.rows()); + + auto && columns = block.getSourceBlock().getColumns(); + for (size_t i = 0; i < existing_columns; ++i) + columns[i] = columns[i]->replicate(offsets); + for (size_t pos : right_keys_to_replicate) + columns[pos] = columns[pos]->replicate(offsets); + + block.getSourceBlock().setColumns(columns); + block = ScatteredBlock(std::move(block).getSourceBlock()); } return remaining_block; } diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 2fc115f8345..c114bb957c2 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -1,18 +1,13 @@ #pragma once -#include #include #include #include #include #include -#include #include -#include -#include - namespace DB { @@ -24,6 +19,11 @@ extern const int LOGICAL_ERROR; namespace detail { +/// Previously ConcurrentHashJoin used IColumn::scatter method to split input blocks to sub-blocks by hash. +/// To avoid copying of columns, we introduce a new class ScatteredBlock that holds a block and a selector. +/// So now each threads get a copy of the source input block and a selector that tells which rows are meant for the given thread. +/// Selector can be seen as just a list of indexes or rows that belong to the given thread. +/// One optimization is to use a continuous range instead of explicit list of indexes when selector contains all indexes from [L, R). class Selector { public: @@ -31,12 +31,11 @@ public: using Indexes = ColumnUInt64; using IndexesPtr = ColumnUInt64::MutablePtr; - Selector() : Selector(0, 0) { } - /// [begin, end) Selector(size_t begin, size_t end) : data(Range{begin, end}) { } - + Selector() : Selector(0, 0) { } explicit Selector(size_t size) : Selector(0, size) { } + explicit Selector(IndexesPtr && selector_) : data(initializeFromSelector(std::move(selector_))) { } class Iterator @@ -105,6 +104,7 @@ public: } } + /// First selector contains first `num_rows` rows, second selector contains the rest std::pair split(size_t num_rows) { chassert(num_rows <= size()); @@ -179,6 +179,7 @@ private: } +/// Source block + list of selected rows. See detail::Selector for more details. struct ScatteredBlock : private boost::noncopyable { using Selector = detail::Selector; @@ -224,7 +225,7 @@ struct ScatteredBlock : private boost::noncopyable /// Accounts only selected rows size_t rows() const { return selector.size(); } - /// Whether block was scattered, i.e. has non-trivial selector + /// Whether `block` was scattered, i.e. `selector` != [0, block.rows()) bool wasScattered() const { chassert(block); @@ -248,7 +249,7 @@ struct ScatteredBlock : private boost::noncopyable selector = Selector(std::move(new_selector)); } - /// Applies selector to block in place + /// Applies `selector` to the `block` in-place void filterBySelector() { if (!block || !wasScattered()) @@ -266,7 +267,7 @@ struct ScatteredBlock : private boost::noncopyable return; } - /// The general case when selector is non-trivial (likely the result of applying a filter) + /// The general case when `selector` is non-trivial (likely the result of applying a filter) auto columns = block.getColumns(); for (auto & col : columns) col = col->index(selector.getIndexes(), /*limit*/ 0); @@ -274,7 +275,7 @@ struct ScatteredBlock : private boost::noncopyable selector = Selector(block.rows()); } - /// Cut first num_rows rows from block in place and returns block with remaining rows + /// Cut first `num_rows` rows from `block` in place and returns block with remaining rows ScatteredBlock cut(size_t num_rows) { SCOPE_EXIT(filterBySelector()); @@ -293,21 +294,6 @@ struct ScatteredBlock : private boost::noncopyable return remaining; } - void replicate(const IColumn::Offsets & offsets, size_t existing_columns, const std::vector & right_keys_to_replicate) - { - chassert(block); - chassert(offsets.size() == rows()); - - auto && columns = block.getColumns(); - for (size_t i = 0; i < existing_columns; ++i) - columns[i] = columns[i]->replicate(offsets); - for (size_t pos : right_keys_to_replicate) - columns[pos] = columns[pos]->replicate(offsets); - - block.setColumns(columns); - selector = Selector(block.rows()); - } - private: Block block; Selector selector; From 88006f6e0952eb5fda97110124d25b6f3b16bc4c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 21 Oct 2024 21:58:49 +0100 Subject: [PATCH 036/104] impl --- src/Interpreters/HashJoin/HashJoin.cpp | 105 ++++++++++++--------- src/Interpreters/HashJoin/HashJoin.h | 4 +- src/Interpreters/HashJoin/ScatteredBlock.h | 27 ++++++ 3 files changed, 88 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 27179ca5a25..f4777b5f7da 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -30,6 +30,8 @@ #include #include #include +#include "Core/Block.h" +#include "Interpreters/HashJoin/ScatteredBlock.h" #include @@ -85,6 +87,11 @@ Block filterColumnsPresentInSampleBlock(const Block & block, const Block & sampl return filtered_block; } +ScatteredBlock filterColumnsPresentInSampleBlock(const ScatteredBlock & block, const Block & sample_block) +{ + return ScatteredBlock{filterColumnsPresentInSampleBlock(block.getSourceBlock(), sample_block)}; +} + Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, const Names &) { for (const auto & sample_column : sample_block.getColumnsWithTypeAndName()) @@ -104,12 +111,6 @@ Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, JoinCommon::convertColumnToNullable(column); } - // for (const auto & column_name : right_key_names) - // { - // auto & column = block.getByName(column_name).column; - // column = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); - // } - return block; } } @@ -557,7 +558,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) } } - size_t rows = source_block.rows(); + const size_t rows = source_block.rows(); data->rows_to_join += rows; const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right); ColumnPtrMap all_key_columns(right_key_names.size()); @@ -567,7 +568,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } - Block block_to_save = filterColumnsPresentInSampleBlock(source_block.getSourceBlock(), savedBlockSample()); + ScatteredBlock block_to_save = filterColumnsPresentInSampleBlock(source_block, savedBlockSample()); if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); @@ -583,7 +584,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) { tmp_stream = &tmp_data->createStream(right_sample_block); } - tmp_stream->write(block_to_save); + tmp_stream->write(block_to_save.getSourceBlock()); return true; } @@ -595,7 +596,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) if (storage_join_lock) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates"); - assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); + assertBlocksHaveEqualStructure(data->sample_block, block_to_save.getSourceBlock(), "joined block"); size_t min_bytes_to_compress = table_join->crossJoinMinBytesToCompress(); size_t min_rows_to_compress = table_join->crossJoinMinRowsToCompress(); @@ -609,12 +610,10 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) have_compressed = true; } - /// In case of scattered block we account proportional share of the source block bytes. - /// For not scattered columns it will be trivial (bytes * N / N) calculation. - data->blocks_allocated_size += block_to_save.rows() ? block_to_save.allocatedBytes() * rows / block_to_save.rows() : 0; + data->blocks_allocated_size += block_to_save.allocatedBytes(); doDebugAsserts(); data->blocks.emplace_back(std::move(block_to_save)); - Block * stored_block = &data->blocks.back(); + auto * stored_block = &data->blocks.back(); doDebugAsserts(); if (rows) @@ -679,7 +678,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) map, key_columns, key_sizes[onexpr_idx], - stored_block, + &stored_block->getSourceBlock(), source_block.getSelector(), null_map, join_mask_col.getData(), @@ -687,7 +686,8 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) is_inserted); if (flag_per_row) - used_flags->reinit, MapsAll>>(stored_block); + used_flags->reinit, MapsAll>>( + &stored_block->getSourceBlock()); else if (is_inserted) /// Number of buckets + 1 value from zero storage used_flags->reinit, MapsAll>>(size + 1); @@ -696,14 +696,16 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) if (!flag_per_row && save_nullmap && is_inserted) { - data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes(); - data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); + data->blocks_nullmaps_allocated_size + += null_map_holder->size() ? null_map_holder->allocatedBytes() * rows / null_map_holder->size() : 0; + data->blocks_nullmaps.emplace_back(&stored_block->getSourceBlock(), null_map_holder); } if (!flag_per_row && not_joined_map && is_inserted) { - data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes(); - data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); + data->blocks_nullmaps_allocated_size + += not_joined_map->size() ? not_joined_map->allocatedBytes() * rows / not_joined_map->size() : 0; + data->blocks_nullmaps.emplace_back(&stored_block->getSourceBlock(), std::move(not_joined_map)); } if (!flag_per_row && !is_inserted) @@ -861,7 +863,7 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) } }; - for (const Block & block_right : data->blocks) + for (const auto & block_right : data->blocks) { ++block_number; if (block_number < start_right_block) @@ -869,9 +871,12 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) /// The following statement cannot be substituted with `process_right_block(!have_compressed ? block_right : block_right.decompress())` /// because it will lead to copying of `block_right` even if its branch is taken (because common type of `block_right` and `block_right.decompress()` is `Block`). if (!have_compressed) - process_right_block(block_right); + process_right_block(block_right.getSourceBlock()); else - process_right_block(block_right.decompress()); + { + chassert(!block_right.wasScattered()); /// Compression only happens for cross join + process_right_block(block_right.getSourceBlock().decompress()); + } if (rows_added > max_joined_block_rows) { @@ -1221,14 +1226,14 @@ private: std::any position; std::optional nulls_position; - std::optional used_position; + std::optional used_position; - size_t fillColumnsFromData(const BlocksList & blocks, MutableColumns & columns_right) + size_t fillColumnsFromData(const HashJoin::ScatteredBlocksList & blocks, MutableColumns & columns_right) { if (!position.has_value()) - position = std::make_any(blocks.begin()); + position = std::make_any(blocks.begin()); - auto & block_it = std::any_cast(position); + auto & block_it = std::any_cast(position); auto end = blocks.end(); size_t rows_added = 0; @@ -1288,11 +1293,11 @@ private: for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it) { - const Block & mapped_block = *it; + const auto & mapped_block = *it; for (size_t row = 0; row < mapped_block.rows(); ++row) { - if (!parent.isUsed(&mapped_block, row)) + if (!parent.isUsed(&mapped_block.getSourceBlock(), row)) { for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum) { @@ -1418,16 +1423,24 @@ void HashJoin::reuseJoinedData(const HashJoin & join) } } -BlocksList HashJoin::releaseJoinedBlocks(bool restructure) +BlocksList HashJoin::releaseJoinedBlocks(bool restructure [[maybe_unused]]) { LOG_TRACE( log, "{}Join data is being released, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount()); - BlocksList right_blocks = std::move(data->blocks); + auto extract_source_blocks = [](ScatteredBlocksList && blocks) + { + BlocksList result; + for (auto & block : blocks) + result.emplace_back(std::move(block).getSourceBlock()); + return result; + }; + + ScatteredBlocksList right_blocks = std::move(data->blocks); if (!restructure) { data.reset(); - return right_blocks; + return extract_source_blocks(std::move(right_blocks)); } data->maps.clear(); @@ -1441,7 +1454,7 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure) if (!right_blocks.empty()) { positions.reserve(right_sample_block.columns()); - const Block & tmp_block = *right_blocks.begin(); + const Block & tmp_block = right_blocks.begin()->getSourceBlock(); for (const auto & sample_column : right_sample_block) { positions.emplace_back(tmp_block.getPositionByName(sample_column.name)); @@ -1449,12 +1462,12 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure) } } - for (Block & saved_block : right_blocks) + for (ScatteredBlock & saved_block : right_blocks) { Block restored_block; for (size_t i = 0; i < positions.size(); ++i) { - auto & column = saved_block.getByPosition(positions[i]); + auto & column = saved_block.getSourceBlock().getByPosition(positions[i]); correctNullabilityInplace(column, is_nullable[i]); restored_block.insert(column); } @@ -1519,7 +1532,6 @@ bool HashJoin::isUsed(const Block * block_ptr, size_t row_idx) const return used_flags->getUsedSafe(block_ptr, row_idx); } - bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const { if (!table_join_->oneDisjunct()) @@ -1538,7 +1550,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged."); else { - auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) + auto merge_rows_into_one_block = [&](ScatteredBlocksList & blocks, RowRefList & rows_ref) { auto it = rows_ref.begin(); if (it.ok()) @@ -1550,7 +1562,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { return; } - auto & block = blocks.back(); + auto & block = blocks.back().getSourceBlock(); size_t start_row = block.rows(); for (; it.ok(); ++it) { @@ -1567,23 +1579,22 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) } }; - auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map) + auto visit_rows_map = [&](ScatteredBlocksList & blocks, MapsAll & rows_map) { switch (data->type) { - #define M(TYPE) \ - case Type::TYPE: \ - {\ - rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \ - break; \ - } +#define M(TYPE) \ + case Type::TYPE: { \ + rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \ + break; \ + } APPLY_FOR_JOIN_VARIANTS(M) - #undef M +#undef M default: break; } }; - BlocksList sorted_blocks; + ScatteredBlocksList sorted_blocks; visit_rows_map(sorted_blocks, map); doDebugAsserts(); data->blocks.swap(sorted_blocks); diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index a76c60aab5a..59cc495e72f 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -337,6 +337,8 @@ public: using RawBlockPtr = const Block *; using BlockNullmapList = std::deque>; + using ScatteredBlocksList = std::list; + struct RightTableData { Type type = Type::EMPTY; @@ -344,7 +346,7 @@ public: std::vector maps; Block sample_block; /// Block as it would appear in the BlockList - BlocksList blocks; /// Blocks of "right" table. + ScatteredBlocksList blocks; /// Blocks of "right" table. BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index c114bb957c2..da4a19ceafb 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -6,6 +6,9 @@ #include #include +#include +#include + #include namespace DB @@ -225,6 +228,30 @@ struct ScatteredBlock : private boost::noncopyable /// Accounts only selected rows size_t rows() const { return selector.size(); } + /// In case of scattered block we account proportional share of the source block bytes. + /// For not scattered columns it will be trivial (bytes * N / N) calculation. + size_t allocatedBytes() const { return block.rows() ? block.allocatedBytes() * rows() / block.rows() : 0; } + + ScatteredBlock shrinkToFit() const + { + if (wasScattered()) + { + LOG_TEST(getLogger("HashJoin"), "shrinkToFit() is not supported for ScatteredBlock because blocks are shared"); + return ScatteredBlock{block}; + } + return ScatteredBlock{block.shrinkToFit()}; + } + + ScatteredBlock compress() const + { + chassert(!wasScattered()); + if (wasScattered()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compress scattered block"); + return ScatteredBlock{block.compress()}; + } + + const auto & getByPosition(size_t i) const { return block.getByPosition(i); } + /// Whether `block` was scattered, i.e. `selector` != [0, block.rows()) bool wasScattered() const { From dd4246725066804b0a5c50a3e6d28f114efb6c4f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 22 Oct 2024 13:00:14 +0100 Subject: [PATCH 037/104] better --- src/Interpreters/HashJoin/HashJoin.cpp | 9 +++------ src/Interpreters/HashJoin/HashJoinMethods.h | 17 ++++++++++++----- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 4 ++-- src/Interpreters/HashJoin/ScatteredBlock.h | 3 +-- 4 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index f4777b5f7da..0d8f12ccb23 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -30,9 +30,6 @@ #include #include #include -#include "Core/Block.h" -#include "Interpreters/HashJoin/ScatteredBlock.h" - #include #include @@ -579,11 +576,11 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) && (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join) || (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join))) { - chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join if (tmp_stream == nullptr) { tmp_stream = &tmp_data->createStream(right_sample_block); } + chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join tmp_stream->write(block_to_save.getSourceBlock()); return true; } @@ -613,7 +610,7 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) data->blocks_allocated_size += block_to_save.allocatedBytes(); doDebugAsserts(); data->blocks.emplace_back(std::move(block_to_save)); - auto * stored_block = &data->blocks.back(); + const auto * stored_block = &data->blocks.back(); doDebugAsserts(); if (rows) @@ -874,7 +871,7 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) process_right_block(block_right.getSourceBlock()); else { - chassert(!block_right.wasScattered()); /// Compression only happens for cross join + chassert(!block_right.wasScattered()); /// Compression only happens for cross join and scattering only for concurrent hash process_right_block(block_right.getSourceBlock().decompress()); } diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index a72087059b3..10fb50a6b83 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -19,7 +19,7 @@ template struct Inserter { static ALWAYS_INLINE bool - insertOne(const HashJoin & join, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + insertOne(const HashJoin & join, HashMap & map, KeyGetter & key_getter, const Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -31,7 +31,8 @@ struct Inserter return false; } - static ALWAYS_INLINE void insertAll(const HashJoin &, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void + insertAll(const HashJoin &, HashMap & map, KeyGetter & key_getter, const Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -45,7 +46,13 @@ struct Inserter } static ALWAYS_INLINE void insertAsof( - HashJoin & join, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) + HashJoin & join, + HashMap & map, + KeyGetter & key_getter, + const Block * stored_block, + size_t i, + Arena & pool, + const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); typename HashMap::mapped_type * time_series_map = &emplace_result.getMapped(); @@ -68,7 +75,7 @@ public: MapsTemplate & maps, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - Block * stored_block, + const Block * stored_block, const ScatteredBlock::Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, @@ -101,7 +108,7 @@ private: HashMap & map, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - Block * stored_block, + const Block * stored_block, const Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 17c8c821bef..5af34689609 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -18,7 +18,7 @@ size_t HashJoinMethods::insertFromBlockImpl( MapsTemplate & maps, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - Block * stored_block, + const Block * stored_block, const ScatteredBlock::Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, @@ -204,7 +204,7 @@ size_t HashJoinMethods::insertFromBlockImplTypeC HashMap & map, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - Block * stored_block, + const Block * stored_block, const Selector & selector, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index da4a19ceafb..8425b3ebd3e 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -244,9 +244,8 @@ struct ScatteredBlock : private boost::noncopyable ScatteredBlock compress() const { - chassert(!wasScattered()); if (wasScattered()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compress scattered block"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compress ScatteredBlock"); return ScatteredBlock{block.compress()}; } From d4e440adb8c70bd46d98cc235ac26321896e18ac Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 22 Oct 2024 14:11:01 +0100 Subject: [PATCH 038/104] bring check back --- src/Interpreters/HashJoin/HashJoin.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 0d8f12ccb23..ee643e45d02 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -385,7 +385,7 @@ size_t HashJoin::getTotalRowCount() const void HashJoin::doDebugAsserts() const { -#if !defined(NDEBUG) && 0 +#ifndef NDEBUG size_t debug_blocks_allocated_size = 0; for (const auto & block : data->blocks) debug_blocks_allocated_size += block.allocatedBytes(); @@ -607,8 +607,8 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) have_compressed = true; } - data->blocks_allocated_size += block_to_save.allocatedBytes(); doDebugAsserts(); + data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); const auto * stored_block = &data->blocks.back(); doDebugAsserts(); From d8f2540b3ca426c98a8562a45b17a684a8d5a381 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 23 Oct 2024 16:37:00 +0100 Subject: [PATCH 039/104] fix --- src/Interpreters/HashJoin/HashJoin.cpp | 12 ++++++------ src/Interpreters/HashJoin/HashJoin.h | 11 +++++++++-- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index ee643e45d02..5e372e9bdcd 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -399,7 +399,7 @@ void HashJoin::doDebugAsserts() const size_t debug_blocks_nullmaps_allocated_size = 0; for (const auto & nullmap : data->blocks_nullmaps) - debug_blocks_nullmaps_allocated_size += nullmap.second->allocatedBytes(); + debug_blocks_nullmaps_allocated_size += nullmap.allocatedBytes(); if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size) throw Exception( @@ -695,14 +695,14 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) { data->blocks_nullmaps_allocated_size += null_map_holder->size() ? null_map_holder->allocatedBytes() * rows / null_map_holder->size() : 0; - data->blocks_nullmaps.emplace_back(&stored_block->getSourceBlock(), null_map_holder); + data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); } if (!flag_per_row && not_joined_map && is_inserted) { data->blocks_nullmaps_allocated_size += not_joined_map->size() ? not_joined_map->allocatedBytes() * rows / not_joined_map->size() : 0; - data->blocks_nullmaps.emplace_back(&stored_block->getSourceBlock(), std::move(not_joined_map)); + data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } if (!flag_per_row && !is_inserted) @@ -1347,10 +1347,10 @@ private: for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it) { - const auto * block = it->first; + const auto * block = it->block; ConstNullMapPtr nullmap = nullptr; - if (it->second) - nullmap = &assert_cast(*it->second).getData(); + if (it->column) + nullmap = &assert_cast(*it->column).getData(); for (size_t row = 0; row < block->rows(); ++row) { diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 59cc495e72f..64cd5cf5a4e 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -334,8 +334,15 @@ public: using MapsVariant = std::variant; - using RawBlockPtr = const Block *; - using BlockNullmapList = std::deque>; + using RawBlockPtr = const ScatteredBlock *; + struct NullMapHolder + { + size_t allocatedBytes() const { return column->size() ? column->allocatedBytes() * block->rows() / column->size() : 0; } + + RawBlockPtr block; + ColumnPtr column; + }; + using BlockNullmapList = std::deque; using ScatteredBlocksList = std::list; From 69e9dd9b3dff6ba787cb5e3cb44c705ff61228a0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 23 Oct 2024 15:07:49 +0100 Subject: [PATCH 040/104] fix --- src/Interpreters/ConcurrentHashJoin.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index f2d1d1418a4..88559e26c71 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -246,8 +246,13 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std: chassert(res.empty()); res.clear(); res.reserve(dispatched_blocks.size()); - std::ranges::transform( - dispatched_blocks, std::back_inserter(res), [](ScatteredBlock & res_block) { return std::move(res_block).getSourceBlock(); }); + for (auto && res_block : dispatched_blocks) + { + if (res_block.rows()) + res.emplace_back(std::move(res_block).getSourceBlock()); + } + if (res.empty()) + res.emplace_back(dispatched_blocks[0].getSourceBlock()); } void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const From b04e7abd2ec8729f97424d39908a3b82e72f0057 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 24 Oct 2024 14:18:33 +0100 Subject: [PATCH 041/104] stash --- src/Interpreters/HashJoin/AddedColumns.h | 2 +- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index d60ccf0038d..4abac0c09d1 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -72,7 +72,7 @@ public: , left_block(left_block_.getSourceBlock()) , join_on_keys(join_on_keys_) , additional_filter_expression(additional_filter_expression_) - , rows_to_add(left_block.rows()) + , rows_to_add(left_block_.rows()) , join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows()) , output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold()) , join_data_sorted(join.getJoinedData()->sorted) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 5af34689609..5cfb5d469e1 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -412,8 +412,8 @@ size_t HashJoinMethods::joinRightColumns( { if (unlikely(current_offset >= max_joined_block_rows)) { - added_columns.offsets_to_replicate->resize(ind); - added_columns.filter.resize(ind); + added_columns.offsets_to_replicate->resize(i); + added_columns.filter.resize(i); break; } } From d848a3b2bbb656807ace0bd4e32b3a6588519047 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 24 Oct 2024 19:31:13 +0100 Subject: [PATCH 042/104] stash --- src/Interpreters/ConcurrentHashJoin.cpp | 4 +++- src/Interpreters/HashJoin/AddedColumns.h | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 88559e26c71..139e093ad07 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -26,6 +26,7 @@ #include #include #include +#include "Core/Defines.h" #include @@ -238,7 +239,8 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std: std::shared_ptr none_extra_block; auto & hash_join = hash_joins[i]; auto & dispatched_block = dispatched_blocks[i]; - hash_join->data->joinBlock(dispatched_block, none_extra_block); + if ((i == 0 && block.rows() == 0) || dispatched_block.rows()) + hash_join->data->joinBlock(dispatched_block, none_extra_block); if (none_extra_block && !none_extra_block->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 4abac0c09d1..3e6a29da6c3 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -167,7 +167,7 @@ public: return; /// Do not allow big allocations when user set max_joined_block_rows to huge value - size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); + size_t reserve_size = std::min(max_joined_block_rows, rows_to_add * 2); /// rows_to_add if (need_replicate) /// Reserve 10% more space for columns, because some rows can be repeated @@ -226,7 +226,7 @@ private: void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) { columns.push_back(src_column.column->cloneEmpty()); - columns.back()->reserve(src_column.column->size()); + columns.back()->reserve(rows_to_add); type_name.emplace_back(src_column.type, src_column.name, qualified_name); } From c2abeca568a7118f4638e7de73f4655736b2e72a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 24 Oct 2024 19:35:17 +0100 Subject: [PATCH 043/104] stash --- src/Interpreters/HashJoin/HashJoin.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 5e372e9bdcd..a3773c0fe62 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -996,6 +996,7 @@ void HashJoin::checkTypesOfKeys(const Block & block) const void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { + LOG_DEBUG(&Poco::Logger::get("debug"), "block.rows()={}", block.rows()); if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); @@ -1062,6 +1063,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) { + LOG_DEBUG(&Poco::Logger::get("debug"), "block.rows()={}", block.rows()); if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); From 292800eb0a198b9d8dd89e864028dc9c95039213 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 25 Oct 2024 13:01:18 +0100 Subject: [PATCH 044/104] stash --- src/Interpreters/ConcurrentHashJoin.cpp | 11 ++++------- src/Interpreters/HashJoin/HashJoin.cpp | 2 -- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 139e093ad07..6ebf72bd33e 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -26,7 +26,6 @@ #include #include #include -#include "Core/Defines.h" #include @@ -239,7 +238,7 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std: std::shared_ptr none_extra_block; auto & hash_join = hash_joins[i]; auto & dispatched_block = dispatched_blocks[i]; - if ((i == 0 && block.rows() == 0) || dispatched_block.rows()) + if (i == 0 || dispatched_block.rows()) hash_join->data->joinBlock(dispatched_block, none_extra_block); if (none_extra_block && !none_extra_block->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); @@ -248,13 +247,11 @@ void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std: chassert(res.empty()); res.clear(); res.reserve(dispatched_blocks.size()); - for (auto && res_block : dispatched_blocks) + for (size_t i = 0; i < dispatched_blocks.size(); ++i) { - if (res_block.rows()) - res.emplace_back(std::move(res_block).getSourceBlock()); + if (i == 0 || dispatched_blocks[i].rows()) + res.emplace_back(std::move(dispatched_blocks[i]).getSourceBlock()); } - if (res.empty()) - res.emplace_back(dispatched_blocks[0].getSourceBlock()); } void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index a3773c0fe62..5e372e9bdcd 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -996,7 +996,6 @@ void HashJoin::checkTypesOfKeys(const Block & block) const void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { - LOG_DEBUG(&Poco::Logger::get("debug"), "block.rows()={}", block.rows()); if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); @@ -1063,7 +1062,6 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) { - LOG_DEBUG(&Poco::Logger::get("debug"), "block.rows()={}", block.rows()); if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); From 5f93afede52bc7ce4672b3ac59d4776590a190cc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 25 Oct 2024 13:20:00 +0100 Subject: [PATCH 045/104] stash --- src/Interpreters/HashJoin/HashJoin.cpp | 6 ++---- src/Interpreters/HashJoin/HashJoin.h | 2 +- src/Interpreters/HashJoin/ScatteredBlock.h | 2 -- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 5e372e9bdcd..526ac456f7f 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -693,16 +693,14 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits) if (!flag_per_row && save_nullmap && is_inserted) { - data->blocks_nullmaps_allocated_size - += null_map_holder->size() ? null_map_holder->allocatedBytes() * rows / null_map_holder->size() : 0; data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); + data->blocks_nullmaps_allocated_size += data->blocks_nullmaps.back().allocatedBytes(); } if (!flag_per_row && not_joined_map && is_inserted) { - data->blocks_nullmaps_allocated_size - += not_joined_map->size() ? not_joined_map->allocatedBytes() * rows / not_joined_map->size() : 0; data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); + data->blocks_nullmaps_allocated_size += data->blocks_nullmaps.back().allocatedBytes(); } if (!flag_per_row && !is_inserted) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 64cd5cf5a4e..9e42d58e6b0 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -337,7 +337,7 @@ public: using RawBlockPtr = const ScatteredBlock *; struct NullMapHolder { - size_t allocatedBytes() const { return column->size() ? column->allocatedBytes() * block->rows() / column->size() : 0; } + size_t allocatedBytes() const { return !column->empty() ? column->allocatedBytes() * block->rows() / column->size() : 0; } RawBlockPtr block; ColumnPtr column; diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 8425b3ebd3e..d8077c30c53 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -254,13 +254,11 @@ struct ScatteredBlock : private boost::noncopyable /// Whether `block` was scattered, i.e. `selector` != [0, block.rows()) bool wasScattered() const { - chassert(block); return selector.size() != block.rows(); } const ColumnWithTypeAndName & getByName(const std::string & name) const { - chassert(block); return block.getByName(name); } From b04b332d8798396a5fbb1162c726dcb14b96e8cd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 30 Oct 2024 23:20:41 +0100 Subject: [PATCH 046/104] support max_joined_block_rows --- src/Interpreters/ConcurrentHashJoin.cpp | 47 ++++++++++++------- src/Interpreters/ConcurrentHashJoin.h | 8 ++-- src/Interpreters/HashJoin/HashJoin.cpp | 7 +-- src/Interpreters/HashJoin/HashJoin.h | 2 +- src/Interpreters/HashJoin/ScatteredBlock.h | 9 ++++ src/Interpreters/IJoin.h | 13 +++-- .../Transforms/JoiningTransform.cpp | 10 +++- src/Processors/Transforms/JoiningTransform.h | 4 ++ .../02962_max_joined_block_rows.sql | 4 +- 9 files changed, 64 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 6ebf72bd33e..746ce5d4e9f 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -27,7 +27,9 @@ #include #include +#include #include +#include using namespace DB; @@ -123,9 +125,7 @@ ConcurrentHashJoin::ConcurrentHashJoin( auto inner_hash_join = std::make_shared(); inner_hash_join->data = std::make_unique( table_join_, right_sample_block, any_take_last_row_, reserve_size, fmt::format("concurrent{}", idx)); - /// Non zero `max_joined_block_rows` allows to process block partially and return not processed part. - /// TODO: It's not handled properly in ConcurrentHashJoin case, so we set it to 0 to disable this feature. - inner_hash_join->data->setMaxJoinedBlockRows(0); + inner_hash_join->data->setMaxJoinedBlockRows(table_join->maxJoinedBlockRows()); hash_joins[idx] = std::move(inner_hash_join); }); } @@ -222,35 +222,50 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) { Blocks res; - std::shared_ptr not_processed; - joinBlock(block, res, not_processed); + ExtraScatteredBlocks extra_blocks; + joinBlock(block, extra_blocks, res); + chassert(!extra_blocks.rows()); block = concatenateBlocks(res); } -void ConcurrentHashJoin::joinBlock(Block & block, std::vector & res, std::shared_ptr & /*not_processed*/) +void ConcurrentHashJoin::joinBlock(Block & block, ExtraScatteredBlocks & extra_blocks, std::vector & res) { - hash_joins[0]->data->materializeColumnsFromLeftBlock(block); + ScatteredBlocks dispatched_blocks; + auto & remaining_blocks = extra_blocks.remaining_blocks; + if (extra_blocks.rows()) + { + dispatched_blocks = std::move(remaining_blocks); + } + else + { + hash_joins[0]->data->materializeColumnsFromLeftBlock(block); + dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); + } - auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); block = {}; + + /// Just in case, should be no-op always + remaining_blocks.resize(slots); + + chassert(res.empty()); + res.clear(); + res.reserve(dispatched_blocks.size()); + for (size_t i = 0; i < dispatched_blocks.size(); ++i) { std::shared_ptr none_extra_block; auto & hash_join = hash_joins[i]; auto & dispatched_block = dispatched_blocks[i]; - if (i == 0 || dispatched_block.rows()) - hash_join->data->joinBlock(dispatched_block, none_extra_block); + if (dispatched_block && (i == 0 || dispatched_block.rows())) + hash_join->data->joinBlock(dispatched_block, remaining_blocks[i]); if (none_extra_block && !none_extra_block->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); } - - chassert(res.empty()); - res.clear(); - res.reserve(dispatched_blocks.size()); for (size_t i = 0; i < dispatched_blocks.size(); ++i) { - if (i == 0 || dispatched_blocks[i].rows()) - res.emplace_back(std::move(dispatched_blocks[i]).getSourceBlock()); + auto & dispatched_block = dispatched_blocks[i]; + if (dispatched_block && (i == 0 || dispatched_block.rows())) + res.emplace_back(std::move(dispatched_block).getSourceBlock()); } } diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 33407045c44..48d487ba433 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -1,13 +1,11 @@ #pragma once -#include #include -#include #include #include #include -#include #include +#include #include #include #include @@ -57,8 +55,8 @@ public: bool alwaysReturnsEmptySet() const override; bool supportParallelJoin() const override { return true; } - bool supportsJoinWithManyResultBlocks() const override { return true; } - void joinBlock(Block & block, std::vector & res, std::shared_ptr & not_processed) override; + bool isScatteredJoin() const override { return true; } + void joinBlock(Block & block, ExtraScatteredBlocks & extra_blocks, std::vector & res) override; IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 526ac456f7f..38175770e35 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1058,7 +1058,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) } } -void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) +void HashJoin::joinBlock(ScatteredBlock & block, ScatteredBlock & remaining_block) { if (!data) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); @@ -1089,7 +1089,6 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_) { - ScatteredBlock remaining_block; if constexpr (std::is_same_v, std::vector>) { remaining_block = HashJoinMethods::joinBlockImpl( @@ -1109,10 +1108,6 @@ void HashJoin::joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type"); } - if (remaining_block.rows()) - not_processed = std::make_shared(std::move(remaining_block).getSourceBlock()); - else - not_processed.reset(); }); chassert(joined); diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 9e42d58e6b0..5fc12879674 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -154,7 +154,7 @@ public: void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; /// Called directly from ConcurrentJoin::joinBlock - void joinBlock(ScatteredBlock & block, ExtraBlockPtr & not_processed); + void joinBlock(ScatteredBlock & block, ScatteredBlock & remaining_block); /// Check joinGet arguments and infer the return type. DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index d8077c30c53..729377f6758 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -325,4 +325,13 @@ private: using ScatteredBlocks = std::vector; +struct ExtraScatteredBlocks +{ + ScatteredBlocks remaining_blocks; + + bool rows() const + { + return std::ranges::any_of(remaining_blocks, [](const auto & bl) { return bl.rows(); }); + } +}; } diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 4bfd43598ec..58cf5254851 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -3,9 +3,10 @@ #include #include -#include -#include #include +#include +#include +#include #include namespace DB @@ -90,13 +91,11 @@ public: /// Could be called from different threads in parallel. virtual void joinBlock(Block & block, std::shared_ptr & not_processed) = 0; - virtual bool supportsJoinWithManyResultBlocks() const { return false; } + virtual bool isScatteredJoin() const { return false; } virtual void joinBlock( - [[maybe_unused]] Block & block, - [[maybe_unused]] std::vector & res, - [[maybe_unused]] std::shared_ptr & not_processed) + [[maybe_unused]] Block & block, [[maybe_unused]] ExtraScatteredBlocks & extra_blocks, [[maybe_unused]] std::vector & res) { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Clone method is not supported for {}", getName()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "joinBlock is not supported for {}", getName()); } /** Set/Get totals for right table diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index ceb95cd7ad1..2862575b541 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -201,8 +201,14 @@ Blocks JoiningTransform::readExecute(Chunk & chunk) auto join_block = [&]() { - if (join->supportsJoinWithManyResultBlocks()) - join->joinBlock(block, res, not_processed); + if (join->isScatteredJoin()) + { + join->joinBlock(block, remaining_blocks, res); + if (remaining_blocks.rows()) + not_processed = std::make_shared(); + else + not_processed.reset(); + } else { join->joinBlock(block, not_processed); diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index c3445cf3e36..d0042983eb5 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -83,6 +84,9 @@ private: bool default_totals; bool initialized = false; + /// Only used with ConcurrentHashJoin + ExtraScatteredBlocks remaining_blocks; + ExtraBlockPtr not_processed; FinishCounterPtr finish_counter; diff --git a/tests/queries/0_stateless/02962_max_joined_block_rows.sql b/tests/queries/0_stateless/02962_max_joined_block_rows.sql index c31ab5e1132..27b2a74b802 100644 --- a/tests/queries/0_stateless/02962_max_joined_block_rows.sql +++ b/tests/queries/0_stateless/02962_max_joined_block_rows.sql @@ -27,11 +27,9 @@ SETTINGS max_joined_block_size_rows = 10; SELECT '--'; --- parallel_hash doen't support max_joined_block_size_rows - SET join_algorithm = 'parallel_hash'; -SELECT max(bs) > 10, b FROM ( +SELECT max(bs) <= 10, b FROM ( SELECT blockSize() as bs, * FROM t1 JOIN t2 ON t1.a = t2.a ) GROUP BY b ORDER BY b From c34de3cda877b2389d9a90be81a1bdddc2a515ca Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 31 Oct 2024 20:00:41 +0100 Subject: [PATCH 047/104] fix tidy --- src/Interpreters/ConcurrentHashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 746ce5d4e9f..d243b223241 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -234,7 +234,7 @@ void ConcurrentHashJoin::joinBlock(Block & block, ExtraScatteredBlocks & extra_b auto & remaining_blocks = extra_blocks.remaining_blocks; if (extra_blocks.rows()) { - dispatched_blocks = std::move(remaining_blocks); + dispatched_blocks.swap(remaining_blocks); } else { From fce3a0463b182b01f8c429b58e172550b27cd3eb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 31 Oct 2024 20:30:05 +0100 Subject: [PATCH 048/104] fix perf test --- src/Interpreters/HashJoin/AddedColumns.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 3e6a29da6c3..8316d5df00f 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -1,4 +1,6 @@ #pragma once + +#include #include #include @@ -167,7 +169,7 @@ public: return; /// Do not allow big allocations when user set max_joined_block_rows to huge value - size_t reserve_size = std::min(max_joined_block_rows, rows_to_add * 2); /// rows_to_add + size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); /// rows_to_add if (need_replicate) /// Reserve 10% more space for columns, because some rows can be repeated From 4e8a96e9c18eb79e3ac5273796664ea6178e5cab Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 2 Nov 2024 15:29:09 +0100 Subject: [PATCH 049/104] squash small blocks before join transforms --- src/QueryPipeline/QueryPipelineBuilder.cpp | 14 +++++++++++--- tests/performance/all_join_opt.xml | 5 +++-- tests/performance/join_append_block.xml | 3 ++- tests/performance/joins_in_memory.xml | 3 ++- tests/performance/one_thousand_joins.xml | 2 +- tests/performance/string_join.xml | 3 ++- 6 files changed, 21 insertions(+), 9 deletions(-) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index be0e17db2a2..463d18ed7a2 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -441,9 +442,12 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Processors processors; for (auto & outport : outports) { + auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 0); + connect(*outport, squashing->getInputs().front()); + processors.emplace_back(squashing); auto adding_joined = std::make_shared(right->getHeader(), join); - connect(*outport, adding_joined->getInputs().front()); - processors.emplace_back(adding_joined); + connect(squashing->getOutputPort(), adding_joined->getInputs().front()); + processors.emplace_back(std::move(adding_joined)); } return processors; }; @@ -497,10 +501,13 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); for (size_t i = 0; i < num_streams; ++i) { + auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 0); + connect(**lit, squashing->getInputs().front()); + auto joining = std::make_shared( left_header, output_header, join, max_block_size, false, default_totals, finish_counter); - connect(**lit, joining->getInputs().front()); + connect(squashing->getOutputPort(), joining->getInputs().front()); connect(**rit, joining->getInputs().back()); if (delayed_root) { @@ -532,6 +539,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (collected_processors) collected_processors->emplace_back(joining); + left->pipe.processors->emplace_back(std::move(squashing)); left->pipe.processors->emplace_back(std::move(joining)); } diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index 0ab9c39f67c..5a4741690c7 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,11 +5,12 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b settings join_algorithm='hash' + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b settings join_algorithm='parallel_hash' SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b DROP TABLE IF EXISTS test DROP TABLE IF EXISTS test1 - \ No newline at end of file + diff --git a/tests/performance/join_append_block.xml b/tests/performance/join_append_block.xml index 15859e95941..3eceb0bbe91 100644 --- a/tests/performance/join_append_block.xml +++ b/tests/performance/join_append_block.xml @@ -1,3 +1,4 @@ - SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number + SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number settings join_algorithm='hash' + SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number settings join_algorithm='parallel_hash' diff --git a/tests/performance/joins_in_memory.xml b/tests/performance/joins_in_memory.xml index 7ff6a803d04..debf533d0e4 100644 --- a/tests/performance/joins_in_memory.xml +++ b/tests/performance/joins_in_memory.xml @@ -13,7 +13,8 @@ SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042) SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 = 20042 - SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0 + SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0 settings join_algorithm='hash' + SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0 settings join_algorithm='parallel_hash' SELECT COUNT() FROM ints l INNER JOIN ints r ON l.i64 = r.i64 WHERE i32 = 20042 SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042) diff --git a/tests/performance/one_thousand_joins.xml b/tests/performance/one_thousand_joins.xml index d7e66bb1c67..d8ac057d967 100644 --- a/tests/performance/one_thousand_joins.xml +++ b/tests/performance/one_thousand_joins.xml @@ -1,6 +1,6 @@ -SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x settings join_algorithm='hash' diff --git a/tests/performance/string_join.xml b/tests/performance/string_join.xml index a7d6837cf6c..b3ecced4600 100644 --- a/tests/performance/string_join.xml +++ b/tests/performance/string_join.xml @@ -9,7 +9,8 @@ 1 - SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null + SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null settings join_algorithm='hash' + SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null settings join_algorithm='parallel_hash' SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (short) FORMAT Null SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (long) FORMAT Null From 74fd7fa01d42fd328dc2905e65719997c2cb20ab Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 3 Nov 2024 08:59:33 -0800 Subject: [PATCH 050/104] Add camelCase alias for anyRespectNulls and anyLastRespectNulls --- src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp index 0b6642bffac..cc0d2cb38c8 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp @@ -223,9 +223,11 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls}); factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("anyRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls}); factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("anyLastRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive); /// Must happen after registering any and anyLast factory.registerNullsActionTransformation("any", "any_respect_nulls"); From 44130d67650334d41f5ef5bd1c0967314f4738fb Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 3 Nov 2024 08:59:56 -0800 Subject: [PATCH 051/104] Add small note in docs for the alias --- docs/en/sql-reference/aggregate-functions/reference/anylast.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index 202d2e9fb10..f5b75e63399 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -18,6 +18,8 @@ anyLast(column) [RESPECT NULLS] :::note Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. + +Alias: anyLastRepectNulls ::: **Returned value** From 2892aa11e55515000b90c2e82922838cda532596 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 3 Nov 2024 16:42:07 +0100 Subject: [PATCH 052/104] squash follow up --- src/Processors/QueryPlan/JoinStep.cpp | 16 +++++++++++----- src/Processors/Transforms/SquashingTransform.cpp | 2 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 ++-- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 018b52a5c68..55cc1020095 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -1,9 +1,10 @@ -#include -#include -#include +#include #include #include -#include +#include +#include +#include +#include #include #include @@ -63,7 +64,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines return joined_pipeline; } - return QueryPipelineBuilder::joinPipelinesRightLeft( + auto ppl = QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), std::move(pipelines[1]), join, @@ -72,6 +73,11 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines max_streams, keep_left_read_in_order, &processors); + + ppl->addSimpleTransform([&](const Block & header) + { return std::make_shared(header, max_block_size / 2, 1_MiB / 2); }); + + return ppl; } bool JoinStep::allowPushDownToRight() const diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 490a57d4e23..0749a305d0e 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -78,7 +78,7 @@ Chunk SimpleSquashingChunksTransform::generate() bool SimpleSquashingChunksTransform::canGenerate() { - return !squashed_chunk.empty(); + return squashed_chunk.hasRows(); } Chunk SimpleSquashingChunksTransform::getRemaining() diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 463d18ed7a2..157577d733d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -442,7 +442,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Processors processors; for (auto & outport : outports) { - auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 0); + auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 1_MiB / 2); connect(*outport, squashing->getInputs().front()); processors.emplace_back(squashing); auto adding_joined = std::make_shared(right->getHeader(), join); @@ -501,7 +501,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); for (size_t i = 0; i < num_streams; ++i) { - auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 0); + auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 1_MiB / 2); connect(**lit, squashing->getInputs().front()); auto joining = std::make_shared( From c6384291e1c96a795020ceca425b1fdff5efe3fe Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 4 Nov 2024 14:31:05 +0100 Subject: [PATCH 053/104] small fix --- src/Interpreters/HashJoin/ScatteredBlock.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 729377f6758..d94497e304b 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -302,8 +302,6 @@ struct ScatteredBlock : private boost::noncopyable /// Cut first `num_rows` rows from `block` in place and returns block with remaining rows ScatteredBlock cut(size_t num_rows) { - SCOPE_EXIT(filterBySelector()); - if (num_rows >= rows()) return ScatteredBlock{Block{}}; From 5b5652a2a6d1b44bfefa676423a93837be26fc17 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 5 Nov 2024 14:00:49 +0100 Subject: [PATCH 054/104] smaller value --- src/Processors/QueryPlan/JoinStep.cpp | 2 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 55cc1020095..7824cd5dbd5 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -75,7 +75,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines &processors); ppl->addSimpleTransform([&](const Block & header) - { return std::make_shared(header, max_block_size / 2, 1_MiB / 2); }); + { return std::make_shared(header, max_block_size / 2, 0); }); return ppl; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 157577d733d..463d18ed7a2 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -442,7 +442,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Processors processors; for (auto & outport : outports) { - auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 1_MiB / 2); + auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 0); connect(*outport, squashing->getInputs().front()); processors.emplace_back(squashing); auto adding_joined = std::make_shared(right->getHeader(), join); @@ -501,7 +501,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); for (size_t i = 0; i < num_streams; ++i) { - auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 1_MiB / 2); + auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 0); connect(**lit, squashing->getInputs().front()); auto joining = std::make_shared( From d5b1b811976e4878af39c18b7727d35750bdbc3c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 5 Nov 2024 14:01:19 +0100 Subject: [PATCH 055/104] fix perf test --- src/Interpreters/ConcurrentHashJoin.cpp | 9 ++++++++- src/Interpreters/ConcurrentHashJoin.h | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index d243b223241..aa1673655be 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -402,9 +402,16 @@ ScatteredBlocks scatterBlocksWithSelector(size_t num_shards, const IColumn::Sele return result; } -ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, Block & from_block) { size_t num_shards = hash_joins.size(); + if (num_shards == 1) + { + ScatteredBlocks res; + res.emplace_back(std::move(from_block)); + return res; + } + IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block); /// With zero-copy approach we won't copy the source columns, but will create a new one with indices. diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 48d487ba433..555a61d4004 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -79,7 +79,7 @@ private: std::mutex totals_mutex; Block totals; - ScatteredBlocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); + ScatteredBlocks dispatchBlock(const Strings & key_columns_names, Block & from_block); }; UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression); From 8e1d85b0b858c2fd5d1509f8c854aa55693b03cf Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 5 Nov 2024 15:07:18 +0100 Subject: [PATCH 056/104] new setting --- src/Core/Settings.cpp | 3 +++ src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 2 ++ src/Processors/QueryPlan/JoinStep.cpp | 10 ++++++++-- src/Processors/QueryPlan/JoinStep.h | 2 ++ src/QueryPipeline/QueryPipelineBuilder.cpp | 5 +++-- src/QueryPipeline/QueryPipelineBuilder.h | 1 + 7 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index df25f395624..9612b4511b2 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -143,6 +143,9 @@ Squash blocks passed to the external table to a specified size in bytes, if bloc )", 0) \ M(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, R"( Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited. +)", 0) \ + M(UInt64, min_joined_block_size_rows, 1024, R"( +Minimum block size for JOIN result (if join algorithm supports it). 0 means unlimited. )", 0) \ M(UInt64, max_insert_threads, 0, R"( The maximum number of threads to execute the `INSERT SELECT` query. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d8c35285210..0932c1d71e9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1886,6 +1886,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getCurrentHeader(), expressions.join, settings[Setting::max_block_size], + 0, max_streams, analysis_result.optimize_read_in_order); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 39c1352c9cf..72492db84d2 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,6 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; + extern const SettingsUInt64 min_joined_block_size_rows; } namespace ErrorCodes @@ -1623,6 +1624,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ right_plan.getCurrentHeader(), std::move(join_algorithm), settings[Setting::max_block_size], + settings[Setting::min_joined_block_size_rows], settings[Setting::max_threads], false /*optimize_read_in_order*/); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 7824cd5dbd5..91e4869f2c1 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -44,9 +44,14 @@ JoinStep::JoinStep( const Header & right_header_, JoinPtr join_, size_t max_block_size_, + size_t min_block_size_, size_t max_streams_, bool keep_left_read_in_order_) - : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) + : join(std::move(join_)) + , max_block_size(max_block_size_) + , min_block_size(min_block_size_) + , max_streams(max_streams_) + , keep_left_read_in_order(keep_left_read_in_order_) { updateInputHeaders({left_header_, right_header_}); } @@ -70,12 +75,13 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines join, *output_header, max_block_size, + min_block_size, max_streams, keep_left_read_in_order, &processors); ppl->addSimpleTransform([&](const Block & header) - { return std::make_shared(header, max_block_size / 2, 0); }); + { return std::make_shared(header, min_block_size, 0); }); return ppl; } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 2793784d633..9222ced2e55 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -18,6 +18,7 @@ public: const Header & right_header_, JoinPtr join_, size_t max_block_size_, + size_t min_block_size_, size_t max_streams_, bool keep_left_read_in_order_); @@ -39,6 +40,7 @@ private: JoinPtr join; size_t max_block_size; + size_t min_block_size; size_t max_streams; bool keep_left_read_in_order; }; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 463d18ed7a2..a9a3b081fac 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -386,6 +386,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe JoinPtr join, const Block & output_header, size_t max_block_size, + size_t min_block_size, size_t max_streams, bool keep_left_read_in_order, Processors * collected_processors) @@ -442,7 +443,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Processors processors; for (auto & outport : outports) { - auto squashing = std::make_shared(right->getHeader(), max_block_size / 2, 0); + auto squashing = std::make_shared(right->getHeader(), min_block_size, 0); connect(*outport, squashing->getInputs().front()); processors.emplace_back(squashing); auto adding_joined = std::make_shared(right->getHeader(), join); @@ -501,7 +502,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); for (size_t i = 0; i < num_streams; ++i) { - auto squashing = std::make_shared(left->getHeader(), max_block_size / 2, 0); + auto squashing = std::make_shared(left->getHeader(), min_block_size, 0); connect(**lit, squashing->getInputs().front()); auto joining = std::make_shared( diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index a9e5b1535c0..34bb62ee0d2 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -126,6 +126,7 @@ public: JoinPtr join, const Block & output_header, size_t max_block_size, + size_t min_block_size, size_t max_streams, bool keep_left_read_in_order, Processors * collected_processors = nullptr); From 174550e1bf23c859c7930bbd145399e6fad46f1e Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 13:28:10 +0330 Subject: [PATCH 057/104] Added "date_time_utc" Signed-off-by: xogoodnow --- .../settings.md | 1 + src/Loggers/OwnJSONPatternFormatter.cpp | 22 +++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 02fa5a8ca58..dd7f22f5c97 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1629,6 +1629,7 @@ You can specify the log format that will be outputted in the console log. Curren ```json { + "date_time_utc": "2024-11-06T09:06:09Z", "date_time": "1650918987.180175", "thread_name": "#1", "thread_id": "254545", diff --git a/src/Loggers/OwnJSONPatternFormatter.cpp b/src/Loggers/OwnJSONPatternFormatter.cpp index 4263ad5925a..3eccb176c1f 100644 --- a/src/Loggers/OwnJSONPatternFormatter.cpp +++ b/src/Loggers/OwnJSONPatternFormatter.cpp @@ -7,12 +7,18 @@ #include #include #include +#include +#include + OwnJSONPatternFormatter::OwnJSONPatternFormatter(Poco::Util::AbstractConfiguration & config) { if (config.has("logger.formatting.names.date_time")) date_time = config.getString("logger.formatting.names.date_time", ""); + if (config.has("logger.formatting.names.date_time_utc")) + date_time_utc= config.getString("logger.formatting.names.date_time_utc", ""); + if (config.has("logger.formatting.names.thread_name")) thread_name = config.getString("logger.formatting.names.thread_name", ""); @@ -41,6 +47,7 @@ OwnJSONPatternFormatter::OwnJSONPatternFormatter(Poco::Util::AbstractConfigurati && logger_name.empty() && message.empty() && source_file.empty() && source_line.empty()) { date_time = "date_time"; + date_time_utc = "date_time_utc"; thread_name = "thread_name"; thread_id = "thread_id"; level = "level"; @@ -62,8 +69,22 @@ void OwnJSONPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ const Poco::Message & msg = msg_ext.base; DB::writeChar('{', wb); + if (!date_time_utc.empty()) + { + writeJSONString(date_time_utc, wb, settings); + DB::writeChar(':', wb); + + DB::writeChar('\"', wb); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); + writeDateTimeTextISO(msg_ext.time_seconds, 0, wb, utc_time_zone); + + DB::writeChar('\"', wb); + print_comma = true; + } + if (!date_time.empty()) { + if (print_comma) DB::writeChar(',', wb); writeJSONString(date_time, wb, settings); DB::writeChar(':', wb); @@ -81,6 +102,7 @@ void OwnJSONPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ print_comma = true; } + if (!thread_name.empty()) { if (print_comma) From 25f73dfb555423ec88e60b5a25f76be927099022 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 13:37:40 +0330 Subject: [PATCH 058/104] Added "date_time_utc" parameter to config file Signed-off-by: xogoodnow --- programs/server/config.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 9807f8c0d5a..98b4f47df74 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -70,12 +70,15 @@ You can specify log format(for now, JSON only). In that case, the console log will be printed in specified format like JSON. For example, as below: + {"date_time":"1650918987.180175","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} + {"date_time_utc":"2024-11-06T09:06:09Z","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} To enable JSON logging support, please uncomment the entire tag below. a) You can modify key names by changing values under tag values inside tag. For example, to change DATE_TIME to MY_DATE_TIME, you can do like: MY_DATE_TIME + date_time_utc b) You can stop unwanted log properties to appear in logs. To do so, you can simply comment out (recommended) that property from this file. For example, if you do not want your log to print query_id, you can comment out only tag. @@ -86,6 +89,7 @@ json date_time + date_time_utc thread_name thread_id level From 9ac9dea447bccac062510f28c5d4d5b915075b58 Mon Sep 17 00:00:00 2001 From: Payam Qorbanpour Date: Thu, 7 Nov 2024 13:48:20 +0330 Subject: [PATCH 059/104] Add 'date_time_utc' format to tests --- .../test_structured_logging_json/test.py | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index a7d048cc4bb..544f81600f6 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -1,8 +1,10 @@ import json +from datetime import datetime from xml.etree import ElementTree as ET import pytest + from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -58,12 +60,21 @@ def validate_log_level(config, logs): return True +def is_valid_utc_datetime(datetime_str): + try: + datetime_obj = datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%S.%fZ") + return datetime_obj.tzinfo is None + except ValueError: + return False + + def validate_log_config_relation(config, logs, config_type): root = ET.fromstring(config) keys_in_config = set() if config_type == "config_no_keys": keys_in_config.add("date_time") + keys_in_config.add("date_time_utc") keys_in_config.add("thread_name") keys_in_config.add("thread_id") keys_in_config.add("level") @@ -85,9 +96,12 @@ def validate_log_config_relation(config, logs, config_type): keys_in_log.add(log_key) if log_key not in keys_in_config: return False - for config_key in keys_in_config: - if config_key not in keys_in_log: - return False + + # Validate the UTC datetime format in "date_time_utc" if it exists + if "date_time_utc" in json_log and not is_valid_utc_datetime( + json_log["date_time_utc"] + ): + return False except ValueError as e: return False return True From 27fc62ae6ac6e67d02836d77d3c83bfa76bdc030 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 14:00:43 +0330 Subject: [PATCH 060/104] Changed custom name for consistency with other example Signed-off-by: xogoodnow --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 98b4f47df74..8ec49d804bd 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -78,7 +78,7 @@ a) You can modify key names by changing values under tag values inside tag. For example, to change DATE_TIME to MY_DATE_TIME, you can do like: MY_DATE_TIME - date_time_utc + MY_UTC_DATE_TIME b) You can stop unwanted log properties to appear in logs. To do so, you can simply comment out (recommended) that property from this file. For example, if you do not want your log to print query_id, you can comment out only tag. From a3bfb57da1d8c007c1aabd43f5145214cbc3bdbf Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 15:27:41 +0330 Subject: [PATCH 061/104] Ran black Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 544f81600f6..4b3f4eb6b96 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -1,12 +1,10 @@ import json from datetime import datetime from xml.etree import ElementTree as ET +from helpers.cluster import ClickHouseCluster import pytest - -from helpers.cluster import ClickHouseCluster - cluster = ClickHouseCluster(__file__) node_all_keys = cluster.add_instance( "node_all_keys", main_configs=["configs/config_all_keys_json.xml"] From b97d78e7f4f9301e3660d31d908b97bba88760e0 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 15:35:17 +0330 Subject: [PATCH 062/104] Third party library must come before local imports (according to isort) Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 4b3f4eb6b96..ae244dde71e 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -1,10 +1,10 @@ import json from datetime import datetime from xml.etree import ElementTree as ET -from helpers.cluster import ClickHouseCluster import pytest +from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node_all_keys = cluster.add_instance( "node_all_keys", main_configs=["configs/config_all_keys_json.xml"] From acafa37e2d48eeee04931b029769cd1bbcac2069 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 15:53:18 +0330 Subject: [PATCH 063/104] Ran black for style check Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index ae244dde71e..bc5f9753f4d 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -5,6 +5,7 @@ from xml.etree import ElementTree as ET import pytest from helpers.cluster import ClickHouseCluster + cluster = ClickHouseCluster(__file__) node_all_keys = cluster.add_instance( "node_all_keys", main_configs=["configs/config_all_keys_json.xml"] From aaa46a95c2160eb74d980bd268f38a664658bdb2 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 7 Nov 2024 18:44:01 +0330 Subject: [PATCH 064/104] Declared the new parameter Signed-off-by: xogoodnow --- src/Loggers/OwnJSONPatternFormatter.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Loggers/OwnJSONPatternFormatter.h b/src/Loggers/OwnJSONPatternFormatter.h index 51827f34b22..ab96c4e5bff 100644 --- a/src/Loggers/OwnJSONPatternFormatter.h +++ b/src/Loggers/OwnJSONPatternFormatter.h @@ -33,6 +33,7 @@ public: private: std::string date_time; + std::string date_time_utc; std::string thread_name; std::string thread_id; std::string level; From 19e6274a403801ebc37e42954dfa98c70d27eb34 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Sun, 10 Nov 2024 15:40:25 +0330 Subject: [PATCH 065/104] Fixed a typo Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index bc5f9753f4d..775d5be202c 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -114,7 +114,7 @@ def validate_logs(logs): return result -def valiade_everything(config, node, config_type): +def validate_everything(config, node, config_type): node.query("SELECT 1") logs = node.grep_in_log("").split("\n") return ( @@ -135,8 +135,8 @@ def test_structured_logging_json_format(start_cluster): ["cat", "/etc/clickhouse-server/config.d/config_no_keys_json.xml"] ) - assert valiade_everything(config_all_keys, node_all_keys, "config_all_keys") == True + assert validate_everything(config_all_keys, node_all_keys, "config_all_keys") == True assert ( - valiade_everything(config_some_keys, node_some_keys, "config_some_keys") == True + validate_everything(config_some_keys, node_some_keys, "config_some_keys") == True ) - assert valiade_everything(config_no_keys, node_no_keys, "config_no_keys") == True + assert validate_everything(config_no_keys, node_no_keys, "config_no_keys") == True From 5258bb6d01642306374e4317c40813a22e51b5b7 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Sun, 10 Nov 2024 16:21:01 +0330 Subject: [PATCH 066/104] Trigger pipeline From 01ca2b6947fcf0494d90b6f819a1f53c9e58aa03 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Sun, 10 Nov 2024 16:34:16 +0330 Subject: [PATCH 067/104] ran black Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 775d5be202c..5fe49b784f2 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -135,8 +135,11 @@ def test_structured_logging_json_format(start_cluster): ["cat", "/etc/clickhouse-server/config.d/config_no_keys_json.xml"] ) - assert validate_everything(config_all_keys, node_all_keys, "config_all_keys") == True assert ( - validate_everything(config_some_keys, node_some_keys, "config_some_keys") == True + validate_everything(config_all_keys, node_all_keys, "config_all_keys") == True + ) + assert ( + validate_everything(config_some_keys, node_some_keys, "config_some_keys") + == True ) assert validate_everything(config_no_keys, node_no_keys, "config_no_keys") == True From 56f6062bd848ee4df0c3bcf9bc2932ec57c52916 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 10 Nov 2024 14:51:19 +0100 Subject: [PATCH 068/104] better --- src/Core/Settings.cpp | 2 +- src/Interpreters/ConcurrentHashJoin.cpp | 6 +++--- src/Interpreters/ConcurrentHashJoin.h | 2 +- src/Planner/PlannerJoinTree.cpp | 4 ++-- src/Processors/QueryPlan/JoinStep.cpp | 14 +++++++------- src/Processors/QueryPlan/JoinStep.h | 4 ++-- src/QueryPipeline/QueryPipelineBuilder.cpp | 6 +++--- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- 8 files changed, 20 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9612b4511b2..dd0a5cd2c52 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -144,7 +144,7 @@ Squash blocks passed to the external table to a specified size in bytes, if bloc M(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, R"( Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited. )", 0) \ - M(UInt64, min_joined_block_size_rows, 1024, R"( + M(UInt64, min_joined_block_size_bytes, 524288, R"( Minimum block size for JOIN result (if join algorithm supports it). 0 means unlimited. )", 0) \ M(UInt64, max_insert_threads, 0, R"( diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index aa1673655be..71b16b9a35d 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -178,7 +178,7 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_l /// (inside different `hash_join`-s) because the block will be shared. Block right_block = hash_joins[0]->data->materializeColumnsFromRightBlock(right_block_); - auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); + auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, std::move(right_block)); size_t blocks_left = 0; for (const auto & block : dispatched_blocks) { @@ -239,7 +239,7 @@ void ConcurrentHashJoin::joinBlock(Block & block, ExtraScatteredBlocks & extra_b else { hash_joins[0]->data->materializeColumnsFromLeftBlock(block); - dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); + dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, std::move(block)); } block = {}; @@ -402,7 +402,7 @@ ScatteredBlocks scatterBlocksWithSelector(size_t num_shards, const IColumn::Sele return result; } -ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, Block & from_block) +ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, Block && from_block) { size_t num_shards = hash_joins.size(); if (num_shards == 1) diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 555a61d4004..2266e53f80c 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -79,7 +79,7 @@ private: std::mutex totals_mutex; Block totals; - ScatteredBlocks dispatchBlock(const Strings & key_columns_names, Block & from_block); + ScatteredBlocks dispatchBlock(const Strings & key_columns_names, Block && from_block); }; UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 72492db84d2..19bd2510d4c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,7 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; - extern const SettingsUInt64 min_joined_block_size_rows; + extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes @@ -1624,7 +1624,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ right_plan.getCurrentHeader(), std::move(join_algorithm), settings[Setting::max_block_size], - settings[Setting::min_joined_block_size_rows], + settings[Setting::min_joined_block_size_bytes], settings[Setting::max_threads], false /*optimize_read_in_order*/); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 91e4869f2c1..86bc818d266 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -44,12 +44,12 @@ JoinStep::JoinStep( const Header & right_header_, JoinPtr join_, size_t max_block_size_, - size_t min_block_size_, + size_t min_block_size_bytes_, size_t max_streams_, bool keep_left_read_in_order_) : join(std::move(join_)) , max_block_size(max_block_size_) - , min_block_size(min_block_size_) + , min_block_size_bytes(min_block_size_bytes_) , max_streams(max_streams_) , keep_left_read_in_order(keep_left_read_in_order_) { @@ -69,21 +69,21 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines return joined_pipeline; } - auto ppl = QueryPipelineBuilder::joinPipelinesRightLeft( + auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, - min_block_size, + min_block_size_bytes, max_streams, keep_left_read_in_order, &processors); - ppl->addSimpleTransform([&](const Block & header) - { return std::make_shared(header, min_block_size, 0); }); + pipeline->addSimpleTransform([&](const Block & header) + { return std::make_shared(header, 0, min_block_size_bytes); }); - return ppl; + return pipeline; } bool JoinStep::allowPushDownToRight() const diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 9222ced2e55..bc9b7600510 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -18,7 +18,7 @@ public: const Header & right_header_, JoinPtr join_, size_t max_block_size_, - size_t min_block_size_, + size_t min_block_size_bytes_, size_t max_streams_, bool keep_left_read_in_order_); @@ -40,7 +40,7 @@ private: JoinPtr join; size_t max_block_size; - size_t min_block_size; + size_t min_block_size_bytes; size_t max_streams; bool keep_left_read_in_order; }; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a9a3b081fac..7612af7e3d5 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -386,7 +386,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe JoinPtr join, const Block & output_header, size_t max_block_size, - size_t min_block_size, + size_t min_block_size_bytes, size_t max_streams, bool keep_left_read_in_order, Processors * collected_processors) @@ -443,7 +443,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Processors processors; for (auto & outport : outports) { - auto squashing = std::make_shared(right->getHeader(), min_block_size, 0); + auto squashing = std::make_shared(right->getHeader(), 0, min_block_size_bytes); connect(*outport, squashing->getInputs().front()); processors.emplace_back(squashing); auto adding_joined = std::make_shared(right->getHeader(), join); @@ -502,7 +502,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); for (size_t i = 0; i < num_streams; ++i) { - auto squashing = std::make_shared(left->getHeader(), min_block_size, 0); + auto squashing = std::make_shared(left->getHeader(), 0, min_block_size_bytes); connect(**lit, squashing->getInputs().front()); auto joining = std::make_shared( diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 34bb62ee0d2..312655b7b6d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -126,7 +126,7 @@ public: JoinPtr join, const Block & output_header, size_t max_block_size, - size_t min_block_size, + size_t min_block_size_bytes, size_t max_streams, bool keep_left_read_in_order, Processors * collected_processors = nullptr); From 64244250c20f6b4083319b6f509c7cf3fa0621a9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 10 Nov 2024 15:06:02 +0100 Subject: [PATCH 069/104] fix --- src/Interpreters/HashJoin/ScatteredBlock.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index d94497e304b..729377f6758 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -302,6 +302,8 @@ struct ScatteredBlock : private boost::noncopyable /// Cut first `num_rows` rows from `block` in place and returns block with remaining rows ScatteredBlock cut(size_t num_rows) { + SCOPE_EXIT(filterBySelector()); + if (num_rows >= rows()) return ScatteredBlock{Block{}}; From 0a79fd0c4087b13e3f79e69a3b8c91c1fd48defd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 10 Nov 2024 15:16:25 +0100 Subject: [PATCH 070/104] fix tests --- .../00779_all_right_join_max_block_size.sql | 1 + .../02001_join_on_const_bs_long.sql.j2 | 1 + .../02236_explain_pipeline_join.reference | 19 ++++++++++--------- .../02962_max_joined_block_rows.sql | 2 ++ 4 files changed, 14 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql b/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql index f14b1393b3b..6b93f13d27f 100644 --- a/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql +++ b/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql @@ -1,2 +1,3 @@ +SET min_joined_block_size_bytes = 0; SET max_block_size = 6; SELECT blockSize() bs FROM (SELECT 1 s) js1 ALL RIGHT JOIN (SELECT arrayJoin([2, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3]) s) js2 USING (s) GROUP BY bs ORDER BY bs; diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 1726bcb7062..7a4d0857182 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -7,6 +7,7 @@ CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); +SET min_joined_block_size_bytes = 0; SET max_block_size = 100; SELECT count() == 2222 FROM t1 JOIN t2 ON 1 = 1; diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index 73d0ca8cb5e..ba366d89139 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,17 +1,18 @@ (Expression) ExpressionTransform (Join) - JoiningTransform 2 → 1 - (Expression) - ExpressionTransform - (Limit) - Limit - (ReadFromSystemNumbers) - NumbersRange 0 → 1 - (Expression) - FillingRightJoinSide + SimpleSquashingTransform + JoiningTransform 2 → 1 + (Expression) ExpressionTransform (Limit) Limit (ReadFromSystemNumbers) NumbersRange 0 → 1 + (Expression) + FillingRightJoinSide + ExpressionTransform + (Limit) + Limit + (ReadFromSystemNumbers) + NumbersRange 0 → 1 diff --git a/tests/queries/0_stateless/02962_max_joined_block_rows.sql b/tests/queries/0_stateless/02962_max_joined_block_rows.sql index 27b2a74b802..9edf757c0f7 100644 --- a/tests/queries/0_stateless/02962_max_joined_block_rows.sql +++ b/tests/queries/0_stateless/02962_max_joined_block_rows.sql @@ -8,6 +8,8 @@ CREATE table t2 (a UInt64) ENGINE = Memory; INSERT INTO t2 SELECT number % 2 FROM numbers(10); +SET min_joined_block_size_bytes = 0; + -- block size is always multiple of 5 because we have 5 rows for each key in right table -- we do not split rows corresponding to the same key From 715649900166b1d1b8aaefce215c9a80e6d60f69 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 10 Nov 2024 19:16:19 +0100 Subject: [PATCH 071/104] don't reserve too much --- src/Interpreters/HashJoin/AddedColumns.h | 2 +- tests/performance/hashjoin_with_large_output.xml | 1 + tests/performance/scripts/perf.py | 3 ++- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 8316d5df00f..885c1baca8c 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -169,7 +169,7 @@ public: return; /// Do not allow big allocations when user set max_joined_block_rows to huge value - size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); /// rows_to_add + size_t reserve_size = std::min(max_joined_block_rows, rows_to_add * 2); if (need_replicate) /// Reserve 10% more space for columns, because some rows can be repeated diff --git a/tests/performance/hashjoin_with_large_output.xml b/tests/performance/hashjoin_with_large_output.xml index f4b61c15f82..1eb351255d4 100644 --- a/tests/performance/hashjoin_with_large_output.xml +++ b/tests/performance/hashjoin_with_large_output.xml @@ -9,6 +9,7 @@ settings join_algorithm='hash' + join_algorithm='parallel_hash' join_algorithm='grace_hash' diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index 9931178fcb4..e4a599cc78d 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -478,6 +478,8 @@ for query_index in queries_to_run: client_seconds = time.perf_counter() - start_seconds print(f"client-time\t{query_index}\t{client_seconds}\t{server_seconds}") + median = [statistics.median(t) for t in all_server_times] + print(f"median\t{query_index}\t{median[0]}") # Run additional profiling queries to collect profile data, but only if test times appeared to be different. # We have to do it after normal runs because otherwise it will affect test statistics too much @@ -491,7 +493,6 @@ for query_index in queries_to_run: pvalue = stats.ttest_ind( all_server_times[0], all_server_times[1], equal_var=False ).pvalue - median = [statistics.median(t) for t in all_server_times] # Keep this consistent with the value used in report. Should eventually move # to (median[1] - median[0]) / min(median), which is compatible with "times" # difference we use in report (max(median) / min(median)). From 899702f0569a5f2f3fff6839b9ac2003968de853 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 11 Nov 2024 13:08:48 +0100 Subject: [PATCH 072/104] fix test --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0ff9d0a6833..3e0c17bcf24 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,6 +77,7 @@ static std::initializer_list Date: Mon, 11 Nov 2024 10:58:45 -0800 Subject: [PATCH 073/104] [Docs] Update links to use official Docker image --- docs/en/getting-started/install.md | 2 +- docs/ru/getting-started/install.md | 2 +- docs/zh/getting-started/install.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 6209ef3c8ee..62071ddb722 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -16,7 +16,7 @@ You have four options for getting up and running with ClickHouse: - **[ClickHouse Cloud](https://clickhouse.com/cloud/):** The official ClickHouse as a service, - built by, maintained and supported by the creators of ClickHouse - **[Quick Install](#quick-install):** an easy-to-download binary for testing and developing with ClickHouse - **[Production Deployments](#available-installation-options):** ClickHouse can run on any Linux, FreeBSD, or macOS with x86-64, modern ARM (ARMv8.2-A up), or PowerPC64LE CPU architecture -- **[Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/):** use the official Docker image in Docker Hub +- **[Docker Image](https://hub.docker.com/_/clickhouse):** use the official Docker image in Docker Hub ## ClickHouse Cloud diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 083ddc8c39c..4b4b018697e 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -154,7 +154,7 @@ sudo "clickhouse-client-$LATEST_VERSION/install/doinst.sh" ### Из Docker образа {#from-docker-image} -Для запуска ClickHouse в Docker нужно следовать инструкции на [Docker Hub](https://hub.docker.com/r/clickhouse/clickhouse-server/). Внутри образов используются официальные `deb`-пакеты. +Для запуска ClickHouse в Docker нужно следовать инструкции на [Docker Hub](https://hub.docker.com/_/clickhouse). Внутри образов используются официальные `deb`-пакеты. ### Из единого бинарного файла {#from-single-binary} diff --git a/docs/zh/getting-started/install.md b/docs/zh/getting-started/install.md index 7e4fb6826e4..8a9c4cd1c60 100644 --- a/docs/zh/getting-started/install.md +++ b/docs/zh/getting-started/install.md @@ -132,7 +132,7 @@ sudo "clickhouse-client-$LATEST_VERSION/install/doinst.sh" ### `Docker`安装包 {#from-docker-image} -要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/r/clickhouse/clickhouse-server/)上的指南。它是官方的`deb`安装包。 +要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/_/clickhouse)上的指南。它是官方的`deb`安装包。 ### 其他环境安装包 {#from-other} From 864a8a63dfbe649674366191d1b50586570fb12a Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Tue, 12 Nov 2024 23:38:22 +0330 Subject: [PATCH 074/104] Added a fix Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 5fe49b784f2..5c2f1a1bef8 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -90,17 +90,17 @@ def validate_log_config_relation(config, logs, config_type): length = min(10, len(logs)) for i in range(0, length): json_log = json.loads(logs[i]) - keys_in_log = set() - for log_key in json_log.keys(): - keys_in_log.add(log_key) - if log_key not in keys_in_config: - return False + keys_in_log = set(json_log.keys()) + + if not keys_in_config.issubset(keys_in_log): + return False # Validate the UTC datetime format in "date_time_utc" if it exists if "date_time_utc" in json_log and not is_valid_utc_datetime( json_log["date_time_utc"] ): return False + except ValueError as e: return False return True From bf188495bd8cbf4962951ced5611246071c65b5e Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 13 Nov 2024 12:28:11 +0330 Subject: [PATCH 075/104] Included the new parameter within the config all keys file Signed-off-by: xogoodnow --- .../configs/config_all_keys_json.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml b/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml index f20fda50319..4626a5bcffc 100644 --- a/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml @@ -12,6 +12,7 @@ json DATE_TIME + DATE_TIME_UTC THREAD_NAME THREAD_ID LEVEL From 51b7916baa7b3e89b18134845e03404803a1ee3d Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 13 Nov 2024 12:31:56 +0330 Subject: [PATCH 076/104] Included the DATE_TIME_UTC parameter within the config files Signed-off-by: xogoodnow --- .../test_structured_logging_json/configs/config_json.xml | 1 + .../test_structured_logging_json/configs/config_no_keys_json.xml | 1 + .../configs/config_some_keys_json.xml | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/integration/test_structured_logging_json/configs/config_json.xml b/tests/integration/test_structured_logging_json/configs/config_json.xml index d2e9e284f4e..36f9363b015 100644 --- a/tests/integration/test_structured_logging_json/configs/config_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_json.xml @@ -11,6 +11,7 @@ json DATE_TIME + DATE_TIME_UTC THREAD_NAME THREAD_ID LEVEL diff --git a/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml b/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml index 21d1e9b34ec..b81c5ab3972 100644 --- a/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml @@ -12,6 +12,7 @@ json From 32311c1db9759961af8c5fc9c3e4e1c4fa9848be Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 13 Nov 2024 12:39:02 +0330 Subject: [PATCH 078/104] Added comment for example log output Signed-off-by: xogoodnow --- .../configs/config_all_keys_json.xml | 1 + .../test_structured_logging_json/configs/config_no_keys_json.xml | 1 + .../configs/config_some_keys_json.xml | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml b/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml index 4626a5bcffc..9336695f098 100644 --- a/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_all_keys_json.xml @@ -6,6 +6,7 @@ in specified format like JSON. For example, as below: {"date_time":"1650918987.180175","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} + {"date_time_utc":"2024-11-06T09:06:09Z","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} To enable JSON logging support, just uncomment tag below. --> diff --git a/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml b/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml index b81c5ab3972..e27ef328894 100644 --- a/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_no_keys_json.xml @@ -6,6 +6,7 @@ in specified format like JSON. For example, as below: {"date_time":"1650918987.180175","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} + {"date_time_utc":"2024-11-06T09:06:09Z","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} To enable JSON logging support, just uncomment tag below. --> diff --git a/tests/integration/test_structured_logging_json/configs/config_some_keys_json.xml b/tests/integration/test_structured_logging_json/configs/config_some_keys_json.xml index 7281bfb233d..3a0c1feb55b 100644 --- a/tests/integration/test_structured_logging_json/configs/config_some_keys_json.xml +++ b/tests/integration/test_structured_logging_json/configs/config_some_keys_json.xml @@ -6,6 +6,7 @@ in specified format like JSON. For example, as below: {"date_time":"1650918987.180175","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} + {"date_time_utc":"2024-11-06T09:06:09Z","thread_name":"#1","thread_id":"254545","level":"Trace","query_id":"","logger_name":"BaseDaemon","message":"Received signal 2","source_file":"../base/daemon/BaseDaemon.cpp; virtual void SignalListener::run()","source_line":"192"} To enable JSON logging support, just uncomment tag below. --> From c29be90fd62aa251d5bcb2f008bee55b30b9f8ab Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Nov 2024 11:23:41 +0000 Subject: [PATCH 079/104] Remove excess check in HashJoinMethodsImpl.h --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 5 ----- ...259_join_condition_executed_block_bug.reference | 0 .../03259_join_condition_executed_block_bug.sql | 14 ++++++++++++++ 3 files changed, 14 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03259_join_condition_executed_block_bug.reference create mode 100644 tests/queries/0_stateless/03259_join_condition_executed_block_bug.sql diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 45a766e2df6..250564b5d60 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -529,11 +529,6 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter } right_col_pos += 1; } - if (!executed_block) - { - result_column = ColumnUInt8::create(); - break; - } for (const auto & col_name : required_column_names) { diff --git a/tests/queries/0_stateless/03259_join_condition_executed_block_bug.reference b/tests/queries/0_stateless/03259_join_condition_executed_block_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03259_join_condition_executed_block_bug.sql b/tests/queries/0_stateless/03259_join_condition_executed_block_bug.sql new file mode 100644 index 00000000000..096e0f3835d --- /dev/null +++ b/tests/queries/0_stateless/03259_join_condition_executed_block_bug.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key; +CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 VALUES ('key1', 'a', 1, 1, 2), ('key1', 'b', 2, 3, 2), ('key1', 'c', 3, 2, 1), ('key1', 'd', 4, 7, 2), ('key1', 'e', 5, 5, 5), ('key2', 'a2', 1, 1, 1), ('key4', 'f', 2, 3, 4); +INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); + +SET allow_experimental_join_condition = true; +SET allow_experimental_analyzer = true; + +SELECT t1.* FROM t1 FULL OUTER JOIN t2 ON t1.key = t2.key AND (t1.a = 2 OR indexHint(t2.a = 2)) FORMAT Null +; From 1ccd88e6df2c247aed19404fff0e5272729e99d6 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 14 Nov 2024 13:12:55 +0330 Subject: [PATCH 080/104] By default the "date_time_utc" is not included in the log Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 5c2f1a1bef8..fa2dd806f2c 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -73,7 +73,6 @@ def validate_log_config_relation(config, logs, config_type): if config_type == "config_no_keys": keys_in_config.add("date_time") - keys_in_config.add("date_time_utc") keys_in_config.add("thread_name") keys_in_config.add("thread_id") keys_in_config.add("level") From b601541ea5791fc0aec4f0722f4e63782053174d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Nov 2024 14:37:03 +0100 Subject: [PATCH 081/104] Use standard variable --- tests/queries/0_stateless/00429_long_http_bufferization.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00429_long_http_bufferization.sh b/tests/queries/0_stateless/00429_long_http_bufferization.sh index 83a6a4e8043..0b06ea660db 100755 --- a/tests/queries/0_stateless/00429_long_http_bufferization.sh +++ b/tests/queries/0_stateless/00429_long_http_bufferization.sh @@ -98,9 +98,9 @@ check_cli_and_http function cmp_http_compression() { $CLICKHOUSE_CLIENT -q "$(query "$1")" > "${CLICKHOUSE_TMP}"/res0 - ch_url 'compress=1' "$1" | "${CLICKHOUSE_BINARY}"-compressor --decompress > "${CLICKHOUSE_TMP}"/res1 - ch_url "compress=1&buffer_size=$2&wait_end_of_query=0" "$1" | "${CLICKHOUSE_BINARY}"-compressor --decompress > "${CLICKHOUSE_TMP}"/res2 - ch_url "compress=1&buffer_size=$2&wait_end_of_query=1" "$1" | "${CLICKHOUSE_BINARY}"-compressor --decompress > "${CLICKHOUSE_TMP}"/res3 + ch_url 'compress=1' "$1" | "${CLICKHOUSE_COMPRESSOR}" --decompress > "${CLICKHOUSE_TMP}"/res1 + ch_url "compress=1&buffer_size=$2&wait_end_of_query=0" "$1" | "${CLICKHOUSE_COMPRESSOR}" --decompress > "${CLICKHOUSE_TMP}"/res2 + ch_url "compress=1&buffer_size=$2&wait_end_of_query=1" "$1" | "${CLICKHOUSE_COMPRESSOR}" --decompress > "${CLICKHOUSE_TMP}"/res3 cmp "${CLICKHOUSE_TMP}"/res0 "${CLICKHOUSE_TMP}"/res1 cmp "${CLICKHOUSE_TMP}"/res1 "${CLICKHOUSE_TMP}"/res2 cmp "${CLICKHOUSE_TMP}"/res1 "${CLICKHOUSE_TMP}"/res3 From 236b09989df8490f9f82c689322260a8ba298a90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Nov 2024 15:01:16 +0100 Subject: [PATCH 082/104] Adapt some test to run in non-CI configurations --- tests/queries/0_stateless/01018_ip_dictionary_long.sql | 10 +++++----- tests/queries/0_stateless/01162_strange_mutations.sh | 2 +- .../01169_old_alter_partition_isolation_stress.sh | 2 +- .../02526_kv_engine_different_filter_type.sql | 2 +- tests/queries/0_stateless/02971_analyzer_remote_id.sh | 2 +- .../0_stateless/03068_analyzer_distributed_join.sql | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01018_ip_dictionary_long.sql b/tests/queries/0_stateless/01018_ip_dictionary_long.sql index cb8ef223c6f..0fa2d7c9438 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary_long.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary_long.sql @@ -35,7 +35,7 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie cca2 String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100) SETTINGS(dictionary_use_async_executor=1, max_threads=8) @@ -139,7 +139,7 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); @@ -209,7 +209,7 @@ INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.25 CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); @@ -287,7 +287,7 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); @@ -493,7 +493,7 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index db7ec8e0755..c4538aa2bb4 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-shared-merge-tree +# Tags: zookeeper, no-replicated-database, no-shared-merge-tree # Tag no-replicated-database: CREATE AS SELECT is disabled # Tag no-shared-merge-tree -- implemented separate test, just bad substituion here diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh index 404042ab64e..0c1b19c1d5f 100755 --- a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database, no-ordinary-database +# Tags: long, no-replicated-database, no-ordinary-database, zookeeper # shellcheck disable=SC2015 diff --git a/tests/queries/0_stateless/02526_kv_engine_different_filter_type.sql b/tests/queries/0_stateless/02526_kv_engine_different_filter_type.sql index 9a762572cea..11e3732849d 100644 --- a/tests/queries/0_stateless/02526_kv_engine_different_filter_type.sql +++ b/tests/queries/0_stateless/02526_kv_engine_different_filter_type.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, use-rocksdb +-- Tags: zookeeper, no-ordinary-database, use-rocksdb DROP TABLE IF EXISTS 02526_keeper_map; DROP TABLE IF EXISTS 02526_rocksdb; diff --git a/tests/queries/0_stateless/02971_analyzer_remote_id.sh b/tests/queries/0_stateless/02971_analyzer_remote_id.sh index 6d504ce3568..e9de19bcf65 100755 --- a/tests/queries/0_stateless/02971_analyzer_remote_id.sh +++ b/tests/queries/0_stateless/02971_analyzer_remote_id.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="CREATE TABLE ${CLICKHOUSE_DATABASE}.x ENGINE = MergeTree() ORDER BY number AS SELECT * FROM numbers(2)" -${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}', '${CLICKHOUSE_DATABASE}.x') SETTINGS enable_analyzer = 1" 2>&1 \ +${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}:${CLICKHOUSE_PORT_TCP}', '${CLICKHOUSE_DATABASE}.x') SETTINGS enable_analyzer = 1" 2>&1 \ | grep -av "ASan doesn't fully support makecontext/swapcontext functions" diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql index 459c8f5e8ac..4ea0a985d28 100644 --- a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database +-- Tags: no-replicated-database, shard -- Closes: https://github.com/ClickHouse/ClickHouse/issues/6571 SET enable_analyzer=1; From 140bd01427b24f86de413f58b99f66601db6f3bc Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 14 Nov 2024 19:20:30 +0330 Subject: [PATCH 083/104] Corrected the check for time format Signed-off-by: xogoodnow --- .../integration/test_structured_logging_json/test.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index fa2dd806f2c..fe511aa3eaf 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -61,10 +61,14 @@ def validate_log_level(config, logs): def is_valid_utc_datetime(datetime_str): try: - datetime_obj = datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%S.%fZ") - return datetime_obj.tzinfo is None + datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%S.%fZ") + return True except ValueError: - return False + try: + datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%SZ") # Without milliseconds + return True + except ValueError: + return False def validate_log_config_relation(config, logs, config_type): @@ -73,6 +77,7 @@ def validate_log_config_relation(config, logs, config_type): if config_type == "config_no_keys": keys_in_config.add("date_time") + keys_in_config.add("date_time_utc") keys_in_config.add("thread_name") keys_in_config.add("thread_id") keys_in_config.add("level") From 5ae6572cd66486129edc9b23b4949a6695e551fc Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 14 Nov 2024 19:33:32 +0330 Subject: [PATCH 084/104] ran black Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index fe511aa3eaf..fea573910ec 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -65,7 +65,9 @@ def is_valid_utc_datetime(datetime_str): return True except ValueError: try: - datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%SZ") # Without milliseconds + datetime.strptime( + datetime_str, "%Y-%m-%dT%H:%M:%SZ" + ) return True except ValueError: return False From 99e5e550da68868f92c267c95fdab07c220b0776 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Thu, 14 Nov 2024 19:47:30 +0330 Subject: [PATCH 085/104] Omitted comment and reformatted Signed-off-by: xogoodnow --- tests/integration/test_structured_logging_json/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index fea573910ec..7ea3d26ba94 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -65,9 +65,7 @@ def is_valid_utc_datetime(datetime_str): return True except ValueError: try: - datetime.strptime( - datetime_str, "%Y-%m-%dT%H:%M:%SZ" - ) + datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%SZ") return True except ValueError: return False From 1926f6ca4a659e570c258da4ee8a197658dd2a5f Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 14 Nov 2024 23:09:49 +0100 Subject: [PATCH 086/104] clean up EXCEPT clause page and add an example of EXCEPT() --- .../sql-reference/statements/select/except.md | 83 +++++++++++++++++-- 1 file changed, 75 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index 8ba7544d21f..d855248ee54 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -5,9 +5,14 @@ sidebar_label: EXCEPT # EXCEPT Clause -The `EXCEPT` clause returns only those rows that result from the first query without the second. The queries must match the number of columns, order, and type. The result of `EXCEPT` can contain duplicate rows. +The `EXCEPT` clause returns only those rows that result from the first query without the second. -Multiple `EXCEPT` statements are executed left to right if parenthesis are not specified. The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority than the `INTERSECT` clause. +- Both queries must have the same number of columns in the same order and data type. +- The result of `EXCEPT` can contain duplicate rows. Use `EXCEPT DISTINCT` if this is not desirable. +- Multiple `EXCEPT` statements are executed from left to right if parentheses are not specified. +- The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority than the `INTERSECT` clause. + +## Syntax ``` sql SELECT column1 [, column2 ] @@ -19,18 +24,33 @@ EXCEPT SELECT column1 [, column2 ] FROM table2 [WHERE condition] - ``` -The condition could be any expression based on your requirements. +The condition could be any expression based on your requirements. + +Additionally, `EXCEPT()` can be used to exclude columns from a result in the same table, as is possible with BigQuery (Google Cloud), using the following syntax: + +```sql +SELECT column1 [, column2 ] EXCEPT(column3 [, column4]) +FROM table1 +[WHERE condition] +``` ## Examples +The examples in this section demonstrate usage of the `EXCEPT` clause. + +### Filtering Numbers Using the `EXCEPT` Clause + Here is a simple example that returns the numbers 1 to 10 that are _not_ a part of the numbers 3 to 8: Query: ``` sql -SELECT number FROM numbers(1,10) EXCEPT SELECT number FROM numbers(3,6); +SELECT number +FROM numbers(1, 10) +EXCEPT +SELECT number +FROM numbers(3, 6) ``` Result: @@ -44,7 +64,53 @@ Result: └────────┘ ``` -`EXCEPT` and `INTERSECT` can often be used interchangeably with different Boolean logic, and they are both useful if you have two tables that share a common column (or columns). For example, suppose we have a few million rows of historical cryptocurrency data that contains trade prices and volume: +### Excluding Specific Columns Using `EXCEPT()` + +`EXCEPT()` can be used to quickly exclude columns from a result. For instance if we want to select all columns from a table, except a few select columns as shown in the example below: + +Query: + +```sql +SHOW COLUMNS IN system.settings + +SELECT * EXCEPT (default, alias_for, readonly, description) +FROM system.settings +LIMIT 5 +``` + +Result: + +```response + ┌─field───────┬─type─────────────────────────────────────────────────────────────────────┬─null─┬─key─┬─default─┬─extra─┐ + 1. │ alias_for │ String │ NO │ │ ᴺᵁᴸᴸ │ │ + 2. │ changed │ UInt8 │ NO │ │ ᴺᵁᴸᴸ │ │ + 3. │ default │ String │ NO │ │ ᴺᵁᴸᴸ │ │ + 4. │ description │ String │ NO │ │ ᴺᵁᴸᴸ │ │ + 5. │ is_obsolete │ UInt8 │ NO │ │ ᴺᵁᴸᴸ │ │ + 6. │ max │ Nullable(String) │ YES │ │ ᴺᵁᴸᴸ │ │ + 7. │ min │ Nullable(String) │ YES │ │ ᴺᵁᴸᴸ │ │ + 8. │ name │ String │ NO │ │ ᴺᵁᴸᴸ │ │ + 9. │ readonly │ UInt8 │ NO │ │ ᴺᵁᴸᴸ │ │ +10. │ tier │ Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) │ NO │ │ ᴺᵁᴸᴸ │ │ +11. │ type │ String │ NO │ │ ᴺᵁᴸᴸ │ │ +12. │ value │ String │ NO │ │ ᴺᵁᴸᴸ │ │ + └─────────────┴──────────────────────────────────────────────────────────────────────────┴──────┴─────┴─────────┴───────┘ + + ┌─name────────────────────┬─value──────┬─changed─┬─min──┬─max──┬─type────┬─is_obsolete─┬─tier───────┐ +1. │ dialect │ clickhouse │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ Dialect │ 0 │ Production │ +2. │ min_compress_block_size │ 65536 │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ UInt64 │ 0 │ Production │ +3. │ max_compress_block_size │ 1048576 │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ UInt64 │ 0 │ Production │ +4. │ max_block_size │ 65409 │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ UInt64 │ 0 │ Production │ +5. │ max_insert_block_size │ 1048449 │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ UInt64 │ 0 │ Production │ + └─────────────────────────┴────────────┴─────────┴──────┴──────┴─────────┴─────────────┴────────────┘ +``` + +### Using `EXCEPT` and `INTERSECT` with Cryptocurrency Data + +`EXCEPT` and `INTERSECT` can often be used interchangeably with different Boolean logic, and they are both useful if you have two tables that share a common column (or columns). +For example, suppose we have a few million rows of historical cryptocurrency data that contains trade prices and volume: + +Query: ```sql CREATE TABLE crypto_prices @@ -72,6 +138,8 @@ ORDER BY trade_date DESC LIMIT 10; ``` +Result: + ```response ┌─trade_date─┬─crypto_name─┬──────volume─┬────price─┬───market_cap─┬──change_1_day─┐ │ 2020-11-02 │ Bitcoin │ 30771456000 │ 13550.49 │ 251119860000 │ -0.013585099 │ @@ -127,7 +195,7 @@ Result: This means of the four cryptocurrencies we own, only Bitcoin has never dropped below $10 (based on the limited data we have here in this example). -## EXCEPT DISTINCT +### Using `EXCEPT DISTINCT` Notice in the previous query we had multiple Bitcoin holdings in the result. You can add `DISTINCT` to `EXCEPT` to eliminate duplicate rows from the result: @@ -146,7 +214,6 @@ Result: └─────────────┘ ``` - **See Also** - [UNION](union.md#union-clause) From af8d7c0b605b2a48e93d8d722a8fd9fc62cb6379 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 14 Nov 2024 23:14:41 +0100 Subject: [PATCH 087/104] fix style --- docs/en/sql-reference/statements/select/except.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index d855248ee54..717c1f1bd4a 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -30,7 +30,7 @@ The condition could be any expression based on your requirements. Additionally, `EXCEPT()` can be used to exclude columns from a result in the same table, as is possible with BigQuery (Google Cloud), using the following syntax: ```sql -SELECT column1 [, column2 ] EXCEPT(column3 [, column4]) +SELECT column1 [, column2 ] EXCEPT (column3 [, column4]) FROM table1 [WHERE condition] ``` From d55ebb19e0106367387efc6ddb2fc0f903e61107 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Fri, 15 Nov 2024 11:10:31 +0330 Subject: [PATCH 088/104] Re run pipeline From 9bd6f9c6b4c4c02391c98754f0f7b22ff1e57f42 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Nov 2024 10:45:13 +0000 Subject: [PATCH 089/104] Smaller fixups --- .../functions/type-conversion-functions.md | 10 +++--- src/Functions/parseDateTime.cpp | 33 +++++++------------ .../02668_parse_datetime.reference | 13 +++++--- .../0_stateless/02668_parse_datetime.sql | 14 +++++--- ...68_parse_datetime_in_joda_syntax.reference | 14 ++++---- .../02668_parse_datetime_in_joda_syntax.sql | 21 ++++++++---- 6 files changed, 58 insertions(+), 47 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1c92a459e13..5f4c59f5218 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -6791,7 +6791,7 @@ parseDateTime(str[, format[, timezone]]) **Returned value(s)** -Returns DateTime values parsed from input string according to a MySQL style format string. +Return a [DateTime](../data-types/datetime.md) value parsed from the input string according to a MySQL-style format string. **Supported format specifiers** @@ -6840,7 +6840,7 @@ parseDateTimeInJodaSyntax(str[, format[, timezone]]) **Returned value(s)** -Returns DateTime values parsed from input string according to a Joda style format. +Return a [DateTime](../data-types/datetime.md) value parsed from the input string according to a Joda-style format string. **Supported format specifiers** @@ -6885,7 +6885,8 @@ parseDateTime64(str[, format[, timezone]]) **Returned value(s)** -Returns [DateTime64](../data-types/datetime64.md) type values parsed from input string according to a MySQL style format string. +Return a [DateTime64](../data-types/datetime64.md) value parsed from the input string according to a MySQL-style format string. +The precision of the returned value is 6. ## parseDateTime64OrZero @@ -6913,7 +6914,8 @@ parseDateTime64InJodaSyntax(str[, format[, timezone]]) **Returned value(s)** -Returns [DateTime64](../data-types/datetime64.md) type values parsed from input string according to a joda style format string. +Return a [DateTime64](../data-types/datetime64.md) value parsed from the input string according to a Joda-style format string. +The precision of the returned value equal to the number of `S` placeholders in the format string (but at most 6). ## parseDateTime64InJodaSyntaxOrZero diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 9a9a8fd93b4..362a1c30288 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -457,16 +457,16 @@ namespace return {}; } - [[nodiscard]] - VoidOrError setScale(UInt8 scale_, ParseSyntax parse_syntax_) + void setScale(UInt32 scale_, ParseSyntax parse_syntax_) { + /// Because the scale argument for parseDateTime*() is constant, always throw an exception (don't allow continuing to the + /// next row like in other set* functions) if (parse_syntax_ == ParseSyntax::MySQL && scale_ != 6) - RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for scale must be 6 for MySQL parse syntax", std::to_string(scale_)) + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Precision {} is invalid (must be 6)", scale); else if (parse_syntax_ == ParseSyntax::Joda && scale_ > 6) - RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for scale must be in the range [0, 6] for Joda syntax", std::to_string(scale_)) + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Precision {} is invalid (must be [0, 6])", scale); scale = scale_; - return {}; } /// For debug @@ -611,7 +611,6 @@ namespace bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -637,13 +636,13 @@ namespace data_type = std::make_shared(6, time_zone_name); else { + /// The precision of the return type is the number of 'S' placeholders. String format = getFormat(arguments); std::vector instructions = parseFormat(format); - /// How many 'S' characters does the format string contain? - UInt32 s_count = 0; + size_t s_count = 0; for (const auto & instruction : instructions) { - const String fragment = instruction.getFragment(); + const String & fragment = instruction.getFragment(); for (char c : fragment) { if (c == 'S') @@ -654,7 +653,6 @@ namespace if (s_count > 0) break; } - /// Use s_count as DateTime64's scale. data_type = std::make_shared(s_count, time_zone_name); } } @@ -715,25 +713,18 @@ namespace const String format = getFormat(arguments); const std::vector instructions = parseFormat(format); const auto & time_zone = getTimeZone(arguments); - /// Make datetime fit in a cache line. - alignas(64) DateTime datetime; + alignas(64) DateTime datetime; /// Make datetime fit in a cache line. for (size_t i = 0; i < input_rows_count; ++i) { datetime.reset(); + if constexpr (return_type == ReturnType::DateTime64) + datetime.setScale(scale, parse_syntax); + StringRef str_ref = col_str->getDataAt(i); Pos cur = str_ref.data; Pos end = str_ref.data + str_ref.size; bool error = false; - if constexpr (return_type == ReturnType::DateTime64) - { - if (auto result = datetime.setScale(static_cast(scale), parse_syntax); !result.has_value()) - { - const ErrorCodeAndMessage & err = result.error(); - throw Exception(err.error_code, "Invalid scale value: {}, {}", std::to_string(scale), err.error_message); - } - } - for (const auto & instruction : instructions) { if (auto result = instruction.perform(cur, end, datetime); result.has_value()) diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index 1733a9dd4df..8694d668a14 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -274,18 +274,21 @@ select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); set session_timezone = 'UTC'; -- don't randomize the session timezone select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); 1 --- The following is test of parseDateTime64 +select parseDateTime(''); -- { serverError NOT_ENOUGH_SPACE } +-- ------------------------------------------------------------------------------------------------------------------------- +-- Tests for parseDateTime64, these are not systematic + select parseDateTime64(''); -- { serverError NOT_ENOUGH_SPACE } +select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2177-10-09 10:30:10.123'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.118112') = toDateTime64('2021-01-04 23:12:34.118112', 6); 1 select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2021-01-04 23:12:34.118112', 6); 1 -select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s'); -- { serverError CANNOT_PARSE_DATETIME } ---leap years and non-leap years +-- leap vs non-leap years select parseDateTime64('2024-02-29 11:23:34.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2024-02-29 11:23:34.123433', 6); 1 select parseDateTime64('2023-02-29 11:22:33.123433', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } @@ -293,7 +296,7 @@ select parseDateTime64('2024-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = t 1 select parseDateTime64('2023-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2023-02-28 23:22:33.123433', 6); 1 --- Test of parseDateTime64OrNull +-- parseDateTime64OrNull select parseDateTime64OrNull('2021-01-04 23:12:34.118') IS NULL; 1 select parseDateTime64OrNull('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') IS NULL; @@ -302,7 +305,7 @@ select parseDateTime64OrNull('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s') 1 select parseDateTime64OrNull('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f') IS NULL; 1 --- Test of parseDateTime64OrZero +-- parseDateTime64OrZero select parseDateTime64OrZero('2021-01-04 23:12:34.118') = toDateTime64('1970-01-01 00:00:00', 6); 1 select parseDateTime64OrZero('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('1970-01-01 00:00:00', 6); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 1f80fb4d1d8..d7538fdb7cb 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -190,31 +190,35 @@ select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); -- The format string argument is optional set session_timezone = 'UTC'; -- don't randomize the session timezone select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); +select parseDateTime(''); -- { serverError NOT_ENOUGH_SPACE } + +-- ------------------------------------------------------------------------------------------------------------------------- +-- Tests for parseDateTime64, these are not systematic --- The following is test of parseDateTime64 select parseDateTime64(''); -- { serverError NOT_ENOUGH_SPACE } +select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2177-10-09 10:30:10.123'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.118112') = toDateTime64('2021-01-04 23:12:34.118112', 6); select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2021-01-04 23:12:34.118112', 6); -select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s'); -- { serverError CANNOT_PARSE_DATETIME } ---leap years and non-leap years +-- leap vs non-leap years select parseDateTime64('2024-02-29 11:23:34.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2024-02-29 11:23:34.123433', 6); select parseDateTime64('2023-02-29 11:22:33.123433', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2024-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2024-02-28 23:22:33.123433', 6); select parseDateTime64('2023-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2023-02-28 23:22:33.123433', 6); --- Test of parseDateTime64OrNull +-- parseDateTime64OrNull select parseDateTime64OrNull('2021-01-04 23:12:34.118') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f') IS NULL; --- Test of parseDateTime64OrZero +-- parseDateTime64OrZero select parseDateTime64OrZero('2021-01-04 23:12:34.118') = toDateTime64('1970-01-01 00:00:00', 6); select parseDateTime64OrZero('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('1970-01-01 00:00:00', 6); select parseDateTime64OrZero('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s') = toDateTime64('1970-01-01 00:00:00', 6); select parseDateTime64OrZero('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('1970-01-01 00:00:00', 6); +-- ------------------------------------------------------------------------------------------------------------------------- -- { echoOff } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference index d2b940562ff..99cce4aa216 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -360,8 +360,8 @@ select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- set session_timezone = 'UTC'; -- don't randomize the session timezone select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); 1 --- Test timezone and timezone offset for parseDateTimeInJodaSyntax select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-0812'); -- { serverError CANNOT_PARSE_DATETIME } +-- timezone and timezone offset select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-0812', 'yyyy-MM-dd HH:mm:ssZ') = toDateTime64('2024-10-09 18:42:10', 6); 1 select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-08123', 'yyyy-MM-dd HH:mm:ssZZZ'); -- {serverError CANNOT_PARSE_DATETIME} @@ -372,7 +372,9 @@ select parseDateTimeInJodaSyntax('2024-10-09 10:30:10EST', 'yyyy-MM-dd HH:mm:ssz -- incorrect timezone offset and timezone select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('2024-10-09 10:30:10ABCD', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError BAD_ARGUMENTS } --- The following is test of parseDateTime64InJodaSyntax[OrNull/OrZero] +-- ------------------------------------------------------------------------------------------------------------------------- +-- Tests for parseDateTime64InJodaSyntax, these are not systematic + select parseDateTime64InJodaSyntax('', '') = toDateTime64('1970-01-01 00:00:00', 0); 1 select parseDateTime64InJodaSyntax('2177-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS'); -- { serverError CANNOT_PARSE_DATETIME } @@ -398,7 +400,7 @@ select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331'); -- { serverError select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331', 'yyyy-MM-dd HH:mm:ss.SSSS') = toDateTime64('2021-01-04 23:12:34.0331', 4); 1 select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331', 'yyyy-MM-dd HH:mm:ss.SS'); -- { serverError CANNOT_PARSE_DATETIME } --- Test timezone and timezone offset for paseDatetTime64InJodaSyntax +-- Timezone and timezone offset select parseDateTime64InJodaSyntax('2024-10-09 10:30:10-0812'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0812', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') = toDateTime64('2024-10-09 18:42:10.123456', 6); 1 @@ -422,7 +424,7 @@ select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0845', 'yyyy-MM-d select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-8000', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456ABCD', 'yyyy-MM-dd HH:mm:ss.SSSSSSz'); -- { serverError BAD_ARGUMENTS } select parseDateTime64InJodaSyntax('2023-02-29 11:22:33Not/Timezone', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError BAD_ARGUMENTS } ---leap years and non-leap years +-- leap vs non-leap years select parseDateTime64InJodaSyntax('2024-02-29 11:23:34America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('2024-02-29 19:23:34', 0); 1 select parseDateTime64InJodaSyntax('2023-02-29 11:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError CANNOT_PARSE_DATETIME } @@ -432,7 +434,7 @@ select parseDateTime64InJodaSyntax('2023-02-28 23:22:33America/Los_Angeles', 'yy 1 select parseDateTime64InJodaSyntax('2024-03-01 00:22:33-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2023-03-01 00:22:33-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } --- Test for parseDateTime64InJodaSyntaxOrNull +-- parseDateTime64InJodaSyntaxOrNull select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') = toDateTime64('2024-10-09 10:30:10.123', 3); 1 select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') = toDateTime64('2024-10-09 10:30:10.123456', 6); @@ -450,7 +452,7 @@ select parseDateTime64InJodaSyntaxOrNull('', '') = toDateTime64('1970-01-01 00:0 1 select parseDateTime64InJodaSyntaxOrNull('2177-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') is NULL; 1 --- Test for parseDateTime64InJodaSyntaxOrZero +-- parseDateTime64InJodaSyntaxOrZero select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') = toDateTime64('2024-10-09 10:30:10.123', 3); 1 select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') = toDateTime64('2024-10-09 10:30:10.123456', 6); diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index a041403139f..f244b8fee35 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -245,8 +245,9 @@ select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- -- The format string argument is optional set session_timezone = 'UTC'; -- don't randomize the session timezone select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); --- Test timezone and timezone offset for parseDateTimeInJodaSyntax select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-0812'); -- { serverError CANNOT_PARSE_DATETIME } + +-- timezone and timezone offset select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-0812', 'yyyy-MM-dd HH:mm:ssZ') = toDateTime64('2024-10-09 18:42:10', 6); select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-08123', 'yyyy-MM-dd HH:mm:ssZZZ'); -- {serverError CANNOT_PARSE_DATETIME} select parseDateTimeInJodaSyntax('2024-10-09 10:30:10EST', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('2024-10-09 15:30:10', 6); @@ -255,7 +256,9 @@ select parseDateTimeInJodaSyntax('2024-10-09 10:30:10EST', 'yyyy-MM-dd HH:mm:ssz select parseDateTimeInJodaSyntax('2024-10-09 10:30:10-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTimeInJodaSyntax('2024-10-09 10:30:10ABCD', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError BAD_ARGUMENTS } --- The following is test of parseDateTime64InJodaSyntax[OrNull/OrZero] +-- ------------------------------------------------------------------------------------------------------------------------- +-- Tests for parseDateTime64InJodaSyntax, these are not systematic + select parseDateTime64InJodaSyntax('', '') = toDateTime64('1970-01-01 00:00:00', 0); select parseDateTime64InJodaSyntax('2177-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('+0000', 'Z') = toDateTime64('1970-01-01 00:00:00', 0); @@ -270,7 +273,8 @@ select parseDateTime64InJodaSyntax('2021/01/04 23:12:34.331', 'yyyy/MM/dd HH:mm: select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331', 'yyyy-MM-dd HH:mm:ss.SSSS') = toDateTime64('2021-01-04 23:12:34.0331', 4); select parseDateTime64InJodaSyntax('2021-01-04 23:12:34.331', 'yyyy-MM-dd HH:mm:ss.SS'); -- { serverError CANNOT_PARSE_DATETIME } --- Test timezone and timezone offset for paseDatetTime64InJodaSyntax + +-- Timezone and timezone offset select parseDateTime64InJodaSyntax('2024-10-09 10:30:10-0812'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0812', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') = toDateTime64('2024-10-09 18:42:10.123456', 6); select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-08123', 'yyyy-MM-dd HH:mm:ss.SSSSSSZZZ'); -- {serverError CANNOT_PARSE_DATETIME} @@ -282,18 +286,21 @@ select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456Australia/Adelaide select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123', 'yyyy-dd-MM HH:mm:ss.SSS') = toDateTime64('2024-09-10 10:30:10.123', 3); select parseDateTime64InJodaSyntax('999999 10-09-202410:30:10', 'SSSSSSSSS dd-MM-yyyyHH:mm:ss'); -- {serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-0845', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') = toDateTime64('2024-10-09 19:15:10.123456', 6); + -- incorrect timezone offset and timezone select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456-8000', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-10-09 10:30:10.123456ABCD', 'yyyy-MM-dd HH:mm:ss.SSSSSSz'); -- { serverError BAD_ARGUMENTS } select parseDateTime64InJodaSyntax('2023-02-29 11:22:33Not/Timezone', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError BAD_ARGUMENTS } ---leap years and non-leap years + +-- leap vs non-leap years select parseDateTime64InJodaSyntax('2024-02-29 11:23:34America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('2024-02-29 19:23:34', 0); select parseDateTime64InJodaSyntax('2023-02-29 11:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2024-02-28 23:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('2024-02-29 07:22:33', 0); select parseDateTime64InJodaSyntax('2023-02-28 23:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('2023-03-01 07:22:33', 0); select parseDateTime64InJodaSyntax('2024-03-01 00:22:33-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64InJodaSyntax('2023-03-01 00:22:33-8000', 'yyyy-MM-dd HH:mm:ssZ'); -- { serverError CANNOT_PARSE_DATETIME } --- Test for parseDateTime64InJodaSyntaxOrNull + +-- parseDateTime64InJodaSyntaxOrNull select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') = toDateTime64('2024-10-09 10:30:10.123', 3); select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') = toDateTime64('2024-10-09 10:30:10.123456', 6); select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123456789', 'yyyy-MM-dd HH:mm:ss.SSSSSSSSS'); -- { serverError CANNOT_PARSE_DATETIME } @@ -303,7 +310,8 @@ select parseDateTime64InJodaSyntaxOrNull('2024-10-09 10:30:10.123', 'yyyy-dd-MM select parseDateTime64InJodaSyntaxOrNull('2023-02-29 11:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') is NULL; select parseDateTime64InJodaSyntaxOrNull('', '') = toDateTime64('1970-01-01 00:00:00', 0); select parseDateTime64InJodaSyntaxOrNull('2177-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') is NULL; --- Test for parseDateTime64InJodaSyntaxOrZero + +-- parseDateTime64InJodaSyntaxOrZero select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') = toDateTime64('2024-10-09 10:30:10.123', 3); select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') = toDateTime64('2024-10-09 10:30:10.123456', 6); select parseDateTime64InJodaSyntaxOrZero('2024-10-09 10:30:10.123456789', 'yyyy-MM-dd HH:mm:ss.SSSSSSSSS'); -- { serverError CANNOT_PARSE_DATETIME } @@ -314,5 +322,6 @@ select parseDateTime64InJodaSyntaxOrZero('wrong value', 'yyyy-dd-MM HH:mm:ss.SSS select parseDateTime64InJodaSyntaxOrZero('2023-02-29 11:22:33America/Los_Angeles', 'yyyy-MM-dd HH:mm:ssz') = toDateTime64('1970-01-01 00:00:00', 0); select parseDateTime64InJodaSyntaxOrZero('', '') = toDateTime64('1970-01-01 00:00:00', 0); select parseDateTime64InJodaSyntaxOrZero('2177-10-09 10:30:10.123', 'yyyy-MM-dd HH:mm:ss.SSS') = toDateTime64('1970-01-01 00:00:00.000', 3); +-- ------------------------------------------------------------------------------------------------------------------------- -- { echoOff } From 864f35dd11799ff017ac1de7f726875513593ccb Mon Sep 17 00:00:00 2001 From: Linh Giang <165205637+linhgiang24@users.noreply.github.com> Date: Fri, 15 Nov 2024 11:30:01 -0700 Subject: [PATCH 090/104] Update aggregatingmergetree.md to include video Added relevant video to the page. --- .../table-engines/mergetree-family/aggregatingmergetree.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 819038ee32c..dc0f8683f11 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -10,6 +10,11 @@ The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-fa You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. +You can see an example of how to use the AggregatingMergeTree and Aggregate functions in the below video: +

+ +
+ The engine processes all columns with the following types: ## [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md) From 183593a5686d301ff5f8cc41bcf133a2ae379bb8 Mon Sep 17 00:00:00 2001 From: Pratima Patel Date: Fri, 15 Nov 2024 21:02:40 -0500 Subject: [PATCH 091/104] Update s3queue.md Updated description for s3queue_polling_min_timeout_ms, s3queue_polling_max_timeout_ms and s3queue_polling_backoff_ms properties --- docs/en/engines/table-engines/integrations/s3queue.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 11fc357d222..89a70420069 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -122,7 +122,7 @@ Default value: `0`. ### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms} -Minimal timeout before next polling (in milliseconds). +Specifies the minimum time, in milliseconds, that ClickHouse waits before making the next polling attempt. Possible values: @@ -132,7 +132,7 @@ Default value: `1000`. ### s3queue_polling_max_timeout_ms {#polling_max_timeout_ms} -Maximum timeout before next polling (in milliseconds). +Defines the maximum time, in milliseconds, that ClickHouse waits before initiating the next polling attempt. Possible values: @@ -142,7 +142,7 @@ Default value: `10000`. ### s3queue_polling_backoff_ms {#polling_backoff_ms} -Polling backoff (in milliseconds). +Determines the additional wait time added to the previous polling interval when no new files are found. The next poll occurs after the sum of the previous interval and this backoff value, or the maximum interval, whichever is lower. Possible values: From 39ae3f4da5ce812d184faa886ec53ebc4871b2ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Nov 2024 02:33:54 +0100 Subject: [PATCH 092/104] Miscellaneous --- src/Common/FailPoint.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index bc1b604d0e5..fc6430c0d64 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -7,7 +7,6 @@ #include #include -#include "config.h" namespace DB { From 7705f9b00995d315214d95137dfbc223f8d8207c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Nov 2024 21:16:15 +0100 Subject: [PATCH 093/104] Fix docs --- docs/en/operations/system-tables/asynchronous_metrics.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index 762d187917c..d506daba95c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -211,7 +211,7 @@ Number of threads in the server of the replicas communication protocol (without The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up. A proxy-indicator of overall system latency and responsiveness. -### LoadAverage_*N* +### LoadAverage*N* The whole system load, averaged with exponential smoothing over 1 minute. The load represents the number of threads across all the processes (the scheduling entities of the OS kernel), that are currently running by CPU or waiting for IO, or ready to run but not being scheduled at this point of time. This number includes all the processes, not only clickhouse-server. The number can be greater than the number of CPU cores, if the system is overloaded, and many processes are ready to run but waiting for CPU or IO. From ada1af57181bf16a809b432abc435068605d1875 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Nov 2024 21:16:38 +0100 Subject: [PATCH 094/104] Fix docs --- src/Common/PipeFDs.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index 50eeda1bbe2..c7ca33bc405 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -10,6 +10,7 @@ #include #include + namespace DB { From ab6e49b930f00384badebd4ecacd326aaeefce9d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 16 Nov 2024 21:42:07 +0100 Subject: [PATCH 095/104] Update src/Interpreters/HashJoin/HashJoin.cpp --- src/Interpreters/HashJoin/HashJoin.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 51385178546..1d0c6f75b8e 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -34,11 +34,6 @@ #include #include -namespace CurrentMetrics -{ -extern const Metric TemporaryFilesForJoin; -} - namespace DB { From df0eec90e8d010d8149682762a9020b22494dab8 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 16 Nov 2024 13:49:51 -0800 Subject: [PATCH 096/104] Fix typos 'of of' and 'The a' in agg functions index.html --- docs/en/sql-reference/aggregate-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 5056ef2c7aa..c297214a49c 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -75,7 +75,7 @@ FROM t_null_big └────────────────────┴─────────────────────┘ ``` -Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. +Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. A `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. ```sql SELECT @@ -110,7 +110,7 @@ GROUP BY v └──────┴─────────┴──────────┘ ``` -And here is an example of of first_value with `RESPECT NULLS` where we can see that NULL inputs are respected and it will return the first value read, whether it's NULL or not: +And here is an example of first_value with `RESPECT NULLS` where we can see that NULL inputs are respected and it will return the first value read, whether it's NULL or not: ```sql SELECT From 7d5fc90b19cc322c96eb7bbf6a1b08349c123266 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 16 Nov 2024 14:43:10 -0800 Subject: [PATCH 097/104] Add lastValueRespectNulls and firstValueRespectNulls aliases --- src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp index fe259f2a0f7..0c08eb9aa7d 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp @@ -224,10 +224,12 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("anyRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("firstValueRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls}); factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("anyLastRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("lastValueRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive); /// Must happen after registering any and anyLast factory.registerNullsActionTransformation("any", "any_respect_nulls"); From 77d288a3e5d63f6c5b7dac0237348bde10939d01 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 16 Nov 2024 15:48:09 -0800 Subject: [PATCH 098/104] Add 03261_any_respect_camelCase_aliases.sql --- ...61_any_respect_camelCase_aliases.reference | 20 +++++++++++++ .../03261_any_respect_camelCase_aliases.sql | 28 +++++++++++++++++++ 2 files changed, 48 insertions(+) create mode 100644 tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference create mode 100644 tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql diff --git a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference new file mode 100644 index 00000000000..1e71c0295cc --- /dev/null +++ b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference @@ -0,0 +1,20 @@ +0 +\N +\N +0 +6 +4 +\N +\N +0 +9 +0 +\N +\N +0 +6 +4 +\N +\N +0 +9 diff --git a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql new file mode 100644 index 00000000000..b09b8b037a0 --- /dev/null +++ b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql @@ -0,0 +1,28 @@ + +-- anyRespectNulls +SELECT anyRespectNulls(number) from numbers(5); +SELECT arrayReduce('anyRespectNulls', [NULL, 10]::Array(Nullable(UInt8))); +SELECT anyRespectNullsMerge(t) FROM (SELECT anyRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyRespectNulls, UInt64)')); +SELECT anyRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); + +-- anyLastRespectNulls +SELECT anyLastRespectNulls(number) from numbers(5); +SELECT arrayReduce('anyLastRespectNulls', [10, NULL]::Array(Nullable(UInt8))); +SELECT anyLastRespectNullsMerge(t) FROM (SELECT anyLastRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyLastRespectNulls, UInt64)')); +SELECT anyLastRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); + +-- firstValueRespectNulls +SELECT firstValueRespectNulls(number) from numbers(5); +SELECT arrayReduce('firstValueRespectNulls', [NULL, 10]::Array(Nullable(UInt8))); +SELECT firstValueRespectNullsMerge(t) FROM (SELECT firstValueRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(firstValueRespectNulls, UInt64)')); +SELECT firstValueRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); + +-- lastValueRespectNulls +SELECT lastValueRespectNulls(number) from numbers(5); +SELECT arrayReduce('lastValueRespectNulls', [10, NULL]::Array(Nullable(UInt8))); +SELECT lastValueRespectNullsMerge(t) FROM (SELECT lastValueRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(lastValueRespectNulls, UInt64)')); +SELECT lastValueRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); From f9431a3150aef6b68a9f0b5e06fd6b3ff3b6cb0e Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 16 Nov 2024 15:49:35 -0800 Subject: [PATCH 099/104] Update docs for any, anyLast, first_value, and last_value --- .../aggregate-functions/reference/any.md | 12 +++++++----- .../aggregate-functions/reference/anylast.md | 12 ++++++------ .../en/sql-reference/window-functions/first_value.md | 6 ++++-- docs/en/sql-reference/window-functions/last_value.md | 6 ++++-- 4 files changed, 21 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index 972263585f2..f7bc375b381 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -21,7 +21,9 @@ Aliases: `any_value`, [`first_value`](../reference/first_value.md). **Returned value** :::note -Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. +By default, the `anyLast` function never returns `NULL`. However, it supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. + +Alias: `anyRespectNulls` ::: :::note @@ -48,11 +50,11 @@ CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); -SELECT any(city) FROM any_nulls; +SELECT any(city), anyRespectNulls(city) FROM any_nulls; ``` ```response -┌─any(city)─┐ -│ Amsterdam │ -└───────────┘ +┌─any(city)─┬─anyRespectNulls(city)─┐ +│ Amsterdam │ ᴺᵁᴸᴸ │ +└───────────┴───────────────────────┘ ``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index fd0bc5c23e1..48cb7124fca 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -18,9 +18,9 @@ anyLast(column) [RESPECT NULLS] :::note -Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the last value passed, regardless of whether it is `NULL` or not. +By default, the `anyLast` function never returns `NULL`. However, it supports the `RESPECT NULLS `modifier after the function name, which will ensure the function selects the last value passed, regardless of whether it is `NULL` or not. -Alias: `anyLastRepectNulls` +Alias: `anyLastRespectNulls` ::: **Returned value** @@ -36,11 +36,11 @@ CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); -SELECT anyLast(city) FROM any_last_nulls; +SELECT anyLast(city), anyLastRespectNulls(city) FROM any_last_nulls; ``` ```response -┌─anyLast(city)─┐ -│ Valencia │ -└───────────────┘ +┌─anyLast(city)─┬─anyLastRespectNulls(city)─┐ +│ Valencia │ ᴺᵁᴸᴸ │ +└───────────────┴───────────────────────────┘ ``` diff --git a/docs/en/sql-reference/window-functions/first_value.md b/docs/en/sql-reference/window-functions/first_value.md index 30c3b1f99dc..c6e978bfc92 100644 --- a/docs/en/sql-reference/window-functions/first_value.md +++ b/docs/en/sql-reference/window-functions/first_value.md @@ -15,7 +15,7 @@ first_value (column_name) [[RESPECT NULLS] | [IGNORE NULLS]] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name -WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column]) ``` Alias: `any`. @@ -23,6 +23,8 @@ Alias: `any`. :::note Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. + +Alias: `firstValueRespectNulls` ::: For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). @@ -48,7 +50,7 @@ CREATE TABLE salaries ) Engine = Memory; -INSERT INTO salaries FORMAT Values +INSERT INTO salaries FORMAT VALUES ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md index dd7f5fa078a..9f3ef8ba4f6 100644 --- a/docs/en/sql-reference/window-functions/last_value.md +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -23,6 +23,8 @@ Alias: `anyLast`. :::note Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. + +Alias: `lastValueRespectNulls` ::: For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). @@ -33,7 +35,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Example** -In this example the `last_value` function is used to find the highest paid footballer from a fictional dataset of salaries of Premier League football players. +In this example the `last_value` function is used to find the lowest paid footballer from a fictional dataset of salaries of Premier League football players. Query: @@ -48,7 +50,7 @@ CREATE TABLE salaries ) Engine = Memory; -INSERT INTO salaries FORMAT Values +INSERT INTO salaries FORMAT VALUES ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), From 082d63904315b71eee534c289bf3ce722a603a8b Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 16 Nov 2024 20:38:18 -0800 Subject: [PATCH 100/104] Empty commit From 3bdd4a5173e0a87318d99d5f3540f61fb9fe560d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Nov 2024 10:10:51 +0000 Subject: [PATCH 101/104] Consistency fixups --- .../aggregate-functions/reference/any.md | 49 ++++++++++++------- .../aggregate-functions/reference/anylast.md | 32 +++++++----- .../AggregateFunctionAnyRespectNulls.cpp | 7 +-- ...61_any_respect_camelCase_aliases.reference | 20 ++++++-- .../03261_any_respect_camelCase_aliases.sql | 38 +++++++++----- 5 files changed, 94 insertions(+), 52 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index f7bc375b381..e7bebd4d460 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -5,7 +5,15 @@ sidebar_position: 102 # any -Selects the first encountered value of a column, ignoring any `NULL` values. +Selects the first encountered value of a column. + +:::warning +As a query can be executed in arbitrary order, the result of this function is non-deterministic. +If you need an arbitrary but deterministic result, use functions [`min`](../reference/min.md) or [`max`](../reference/max.md). +::: + +By default, the function never returns NULL, i.e. ignores NULL values in the input column. +However, if the function is used with the `RESPECT NULLS` modifier, it returns the first value reads no matter if NULL or not. **Syntax** @@ -13,44 +21,47 @@ Selects the first encountered value of a column, ignoring any `NULL` values. any(column) [RESPECT NULLS] ``` -Aliases: `any_value`, [`first_value`](../reference/first_value.md). +Aliases `any(column)` (without `RESPECT NULLS`) +- `any_value` +- [`first_value`](../reference/first_value.md). + +Alias for `any(column) RESPECT NULLS` +- `anyRespectNulls`, `any_respect_nulls` +- `firstValueRespectNulls`, `first_value_respect_nulls` +- `anyValueRespectNulls`, `any_value_respect_nulls` **Parameters** -- `column`: The column name. +- `column`: The column name. **Returned value** -:::note -By default, the `anyLast` function never returns `NULL`. However, it supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. - -Alias: `anyRespectNulls` -::: +The first value encountered. :::note -The return type of the function is the same as the input, except for LowCardinality which is discarded. This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour. -::: - -:::warning -The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate. -To get a determinate result, you can use the [`min`](../reference/min.md) or [`max`](../reference/max.md) function instead of `any`. +The return type of the function is the same as the input, except for LowCardinality which is discarded. +This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). +You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour. ::: **Implementation details** -In some cases, you can rely on the order of execution. This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`. +In some cases, you can rely on the order of execution. +This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`. -When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. +When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. +In other words, each column selected from the table must be used either in keys or inside aggregate functions. +To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. **Example** Query: ```sql -CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; +CREATE TABLE tab (city Nullable(String)) ENGINE=Memory; -INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); +INSERT INTO tab (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); -SELECT any(city), anyRespectNulls(city) FROM any_nulls; +SELECT any(city), anyRespectNulls(city) FROM tab; ``` ```response diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index 48cb7124fca..3d80533e146 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -5,7 +5,15 @@ sidebar_position: 105 # anyLast -Selects the last value encountered, ignoring any `NULL` values by default. The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. +Selects the last encountered value of a column. + +:::warning +As a query can be executed in arbitrary order, the result of this function is non-deterministic. +If you need an arbitrary but deterministic result, use functions [`min`](../reference/min.md) or [`max`](../reference/max.md). +::: + +By default, the function never returns NULL, i.e. ignores NULL values in the input column. +However, if the function is used with the `RESPECT NULLS` modifier, it returns the first value reads no matter if NULL or not. **Syntax** @@ -13,15 +21,15 @@ Selects the last value encountered, ignoring any `NULL` values by default. The r anyLast(column) [RESPECT NULLS] ``` +Alias `anyLast(column)` (without `RESPECT NULLS`) +- [`last_value`](../reference/last_value.md). + +Aliases for `anyLast(column) RESPECT NULLS` +- `anyLastRespectNulls`, `anyLast_respect_nulls` +- `lastValueRespectNulls`, `last_value_respect_nulls` + **Parameters** -- `column`: The column name. - -:::note - -By default, the `anyLast` function never returns `NULL`. However, it supports the `RESPECT NULLS `modifier after the function name, which will ensure the function selects the last value passed, regardless of whether it is `NULL` or not. - -Alias: `anyLastRespectNulls` -::: +- `column`: The column name. **Returned value** @@ -32,11 +40,11 @@ Alias: `anyLastRespectNulls` Query: ```sql -CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; +CREATE TABLE tab (city Nullable(String)) ENGINE=Memory; -INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); +INSERT INTO tab (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); -SELECT anyLast(city), anyLastRespectNulls(city) FROM any_last_nulls; +SELECT anyLast(city), anyLastRespectNulls(city) FROM tab; ``` ```response diff --git a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp index 0c08eb9aa7d..83fc98ada11 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp @@ -221,14 +221,15 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor = {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true}; factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls}); - factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); - factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("anyRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("firstValueRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("anyValueRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive); factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls}); - factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("anyLastRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("lastValueRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive); /// Must happen after registering any and anyLast diff --git a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference index 1e71c0295cc..39f78128b24 100644 --- a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference +++ b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.reference @@ -1,18 +1,28 @@ +anyRespectNulls 0 \N \N 0 6 -4 -\N -\N -0 -9 +firstValueRespectNulls 0 \N \N 0 6 +anyValueRespectNulls +0 +\N +\N +0 +6 +lastValueRespectNulls +4 +\N +\N +0 +9 +anyLastRespectNulls 4 \N \N diff --git a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql index b09b8b037a0..c56f096242c 100644 --- a/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql +++ b/tests/queries/0_stateless/03261_any_respect_camelCase_aliases.sql @@ -1,28 +1,40 @@ +-- Tests aliases of any and anyLast functions --- anyRespectNulls -SELECT anyRespectNulls(number) from numbers(5); +-- aliases of any + +SELECT 'anyRespectNulls'; +SELECT anyRespectNulls(number) FROM numbers(5); SELECT arrayReduce('anyRespectNulls', [NULL, 10]::Array(Nullable(UInt8))); SELECT anyRespectNullsMerge(t) FROM (SELECT anyRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyRespectNulls, UInt64)')); SELECT anyRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); --- anyLastRespectNulls -SELECT anyLastRespectNulls(number) from numbers(5); -SELECT arrayReduce('anyLastRespectNulls', [10, NULL]::Array(Nullable(UInt8))); -SELECT anyLastRespectNullsMerge(t) FROM (SELECT anyLastRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); -SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyLastRespectNulls, UInt64)')); -SELECT anyLastRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); - --- firstValueRespectNulls -SELECT firstValueRespectNulls(number) from numbers(5); +SELECT 'firstValueRespectNulls'; +SELECT firstValueRespectNulls(number) FROM numbers(5); SELECT arrayReduce('firstValueRespectNulls', [NULL, 10]::Array(Nullable(UInt8))); SELECT firstValueRespectNullsMerge(t) FROM (SELECT firstValueRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(firstValueRespectNulls, UInt64)')); SELECT firstValueRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); --- lastValueRespectNulls -SELECT lastValueRespectNulls(number) from numbers(5); +SELECT 'anyValueRespectNulls'; +SELECT anyValueRespectNulls(number) FROM numbers(5); +SELECT arrayReduce('anyValueRespectNulls', [NULL, 10]::Array(Nullable(UInt8))); +SELECT anyValueRespectNullsMerge(t) FROM (SELECT anyValueRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyValueRespectNulls, UInt64)')); +SELECT anyValueRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); + +-- aliases of anyLast + +SELECT 'lastValueRespectNulls'; +SELECT lastValueRespectNulls(number) FROM numbers(5); SELECT arrayReduce('lastValueRespectNulls', [10, NULL]::Array(Nullable(UInt8))); SELECT lastValueRespectNullsMerge(t) FROM (SELECT lastValueRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(lastValueRespectNulls, UInt64)')); SELECT lastValueRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); + +SELECT 'anyLastRespectNulls'; +SELECT anyLastRespectNulls(number) FROM numbers(5); +SELECT arrayReduce('anyLastRespectNulls', [10, NULL]::Array(Nullable(UInt8))); +SELECT anyLastRespectNullsMerge(t) FROM (SELECT anyLastRespectNullsState(NULL::Nullable(UInt8)) as t FROM numbers(5)); +SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(anyLastRespectNulls, UInt64)')); +SELECT anyLastRespectNullsIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) FROM numbers(10); From d0842ee937c61f0a62f87fa7b5bae781edf53acd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 17 Nov 2024 15:36:22 +0100 Subject: [PATCH 102/104] don't squash output for ordinary join for now --- src/Processors/QueryPlan/JoinStep.cpp | 7 +++++-- .../02236_explain_pipeline_join.reference | 19 +++++++++---------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 86bc818d266..d81ce2fda17 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -80,8 +80,11 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines keep_left_read_in_order, &processors); - pipeline->addSimpleTransform([&](const Block & header) - { return std::make_shared(header, 0, min_block_size_bytes); }); + if (join->supportParallelJoin()) + { + pipeline->addSimpleTransform([&](const Block & header) + { return std::make_shared(header, 0, min_block_size_bytes); }); + } return pipeline; } diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index ba366d89139..73d0ca8cb5e 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,18 +1,17 @@ (Expression) ExpressionTransform (Join) - SimpleSquashingTransform - JoiningTransform 2 → 1 - (Expression) + JoiningTransform 2 → 1 + (Expression) + ExpressionTransform + (Limit) + Limit + (ReadFromSystemNumbers) + NumbersRange 0 → 1 + (Expression) + FillingRightJoinSide ExpressionTransform (Limit) Limit (ReadFromSystemNumbers) NumbersRange 0 → 1 - (Expression) - FillingRightJoinSide - ExpressionTransform - (Limit) - Limit - (ReadFromSystemNumbers) - NumbersRange 0 → 1 From a7831c991fbe7c7abd01d43eaa4d6d9071128239 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 17 Nov 2024 17:33:38 +0100 Subject: [PATCH 103/104] Better conditions when cancelling a backup. --- src/Backups/BackupCoordinationStageSync.cpp | 21 ++++++++++++++++----- src/Backups/BackupCoordinationStageSync.h | 3 +++ 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index df5f08091ba..1d7f93398cc 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -685,13 +685,13 @@ void BackupCoordinationStageSync::cancelQueryIfError() { std::lock_guard lock{mutex}; - if (!state.host_with_error) - return; - - exception = state.hosts.at(*state.host_with_error).exception; + if (state.host_with_error) + exception = state.hosts.at(*state.host_with_error).exception; } - chassert(exception); + if (!exception) + return; + process_list_element->cancelQuery(false, exception); state_changed.notify_all(); } @@ -741,6 +741,11 @@ void BackupCoordinationStageSync::cancelQueryIfDisconnectedTooLong() if (!exception) return; + /// In this function we only pass the new `exception` (about that the connection was lost) to `process_list_element`. + /// We don't try to create the 'error' node here (because this function is called from watchingThread() and + /// we don't want the watching thread to try waiting here for retries or a reconnection). + /// Also we don't set the `state.host_with_error` field here because `state.host_with_error` can only be set + /// AFTER creating the 'error' node (see the comment for `State`). process_list_element->cancelQuery(false, exception); state_changed.notify_all(); } @@ -870,6 +875,9 @@ bool BackupCoordinationStageSync::checkIfHostsReachStage(const Strings & hosts, continue; } + if (state.host_with_error) + std::rethrow_exception(state.hosts.at(*state.host_with_error).exception); + if (host_info.finished) throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "{} finished without coming to stage {}", getHostDesc(host), stage_to_wait); @@ -1150,6 +1158,9 @@ bool BackupCoordinationStageSync::checkIfOtherHostsFinish( if ((host == current_host) || host_info.finished) continue; + if (throw_if_error && state.host_with_error) + std::rethrow_exception(state.hosts.at(*state.host_with_error).exception); + String reason_text = reason.empty() ? "" : (" " + reason); String host_status; diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index 879b2422b84..acfd31c05af 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -197,6 +197,9 @@ private: }; /// Information about all the host participating in the current BACKUP or RESTORE operation. + /// This information is read from ZooKeeper. + /// To simplify the programming logic `state` can only be updated AFTER changing corresponding nodes in ZooKeeper + /// (for example, first we create the 'error' node, and only after that we set or read from ZK the `state.host_with_error` field). struct State { std::map hosts; /// std::map because we need to compare states From e00128c71796249436283bbe6e1acdb24b6adc0e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 17 Nov 2024 17:36:22 +0100 Subject: [PATCH 104/104] Increase timeout in test test_long_disconnection_stops_backup. --- .../test_backup_restore_on_cluster/test_cancel_backup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py b/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py index 4ad53acc735..e9e20602d5c 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py +++ b/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py @@ -667,7 +667,7 @@ def test_long_disconnection_stops_backup(): # A backup is expected to fail, but it isn't expected to fail too soon. print(f"Backup failed after {time_to_fail} seconds disconnection") assert time_to_fail > 3 - assert time_to_fail < 35 + assert time_to_fail < 45 # A backup must NOT be stopped if Zookeeper is disconnected shorter than `failure_after_host_disconnected_for_seconds`.