From cc5456c649a4bd1f95321f4320fadfa382236d1b Mon Sep 17 00:00:00 2001 From: cangyin Date: Sat, 6 Apr 2024 19:38:37 +0000 Subject: [PATCH 01/41] Fix projection merge for Collapsing/Replacing/VersionedCollapsing MergeTree --- .../MergeTree/MergeProjectionPartsTask.cpp | 95 +++++++++++ .../MergeTree/MergeProjectionPartsTask.h | 84 ++++++++++ src/Storages/MergeTree/MergeTask.cpp | 157 +++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 25 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 139 +--------------- 7 files changed, 349 insertions(+), 156 deletions(-) create mode 100644 src/Storages/MergeTree/MergeProjectionPartsTask.cpp create mode 100644 src/Storages/MergeTree/MergeProjectionPartsTask.h diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp new file mode 100644 index 00000000000..bdc2ba8b9ca --- /dev/null +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -0,0 +1,95 @@ +#include + +#include +#include + +namespace DB +{ + +bool MergeProjectionPartsTask::executeStep() +{ + auto & current_level_parts = level_parts[current_level]; + auto & next_level_parts = level_parts[next_level]; + + MergeTreeData::MutableDataPartsVector selected_parts; + while (selected_parts.size() < max_parts_to_merge_in_one_level && !current_level_parts.empty()) + { + selected_parts.push_back(std::move(current_level_parts.back())); + current_level_parts.pop_back(); + } + + if (selected_parts.empty()) + { + if (next_level_parts.empty()) + { + LOG_WARNING(log, "There is no projection parts merged"); + + /// Task is finished + return false; + } + current_level = next_level; + ++next_level; + } + else if (selected_parts.size() == 1) + { + if (next_level_parts.empty()) + { + LOG_DEBUG(log, "Merged a projection part in level {}", current_level); + selected_parts[0]->renameTo(projection.name + ".proj", true); + selected_parts[0]->setName(projection.name); + selected_parts[0]->is_temp = false; + new_data_part->addProjectionPart(name, std::move(selected_parts[0])); + + /// Task is finished + return false; + } + else + { + LOG_DEBUG(log, "Forwarded part {} in level {} to next level", selected_parts[0]->name, current_level); + next_level_parts.push_back(std::move(selected_parts[0])); + } + } + else if (selected_parts.size() > 1) + { + // Generate a unique part name + ++block_num; + auto projection_future_part = std::make_shared(); + MergeTreeData::DataPartsVector const_selected_parts( + std::make_move_iterator(selected_parts.begin()), std::make_move_iterator(selected_parts.end())); + projection_future_part->assign(std::move(const_selected_parts)); + projection_future_part->name = fmt::format("{}_{}", projection.name, ++block_num); + projection_future_part->part_info = {"all", 0, 0, 0}; + + MergeTreeData::MergingParams projection_merging_params; + projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; + if (projection.type == ProjectionDescription::Type::Aggregate) + projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; + + LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part->name); + auto tmp_part_merge_task = mutator->mergePartsToTemporaryPart( + projection_future_part, + projection.metadata, + merge_entry, + std::make_unique((*merge_entry)->table_id, projection_future_part, context), + *table_lock_holder, + time_of_merge, + context, + space_reservation, + false, // TODO Do we need deduplicate for projections + {}, + false, // no cleanup + projection_merging_params, + NO_TRANSACTION_PTR, + /* need_prefix */ true, + new_data_part.get(), + ".tmp_proj"); + + next_level_parts.push_back(executeHere(tmp_part_merge_task)); + next_level_parts.back()->is_temp = true; + } + + /// Need execute again + return true; +} + +} diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.h b/src/Storages/MergeTree/MergeProjectionPartsTask.h new file mode 100644 index 00000000000..47cafe01151 --- /dev/null +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class MergeProjectionPartsTask : public IExecutableTask +{ +public: + + MergeProjectionPartsTask( + String name_, + MergeTreeData::MutableDataPartsVector && parts_, + const ProjectionDescription & projection_, + size_t & block_num_, + ContextPtr context_, + TableLockHolder * table_lock_holder_, + MergeTreeDataMergerMutator * mutator_, + MergeListEntry * merge_entry_, + time_t time_of_merge_, + MergeTreeData::MutableDataPartPtr new_data_part_, + ReservationSharedPtr space_reservation_) + : name(std::move(name_)) + , parts(std::move(parts_)) + , projection(projection_) + , block_num(block_num_) + , context(context_) + , table_lock_holder(table_lock_holder_) + , mutator(mutator_) + , merge_entry(merge_entry_) + , time_of_merge(time_of_merge_) + , new_data_part(new_data_part_) + , space_reservation(space_reservation_) + , log(getLogger("MergeProjectionPartsTask")) + { + LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); + level_parts[current_level] = std::move(parts); + } + + void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + StorageID getStorageID() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + Priority getPriority() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + String getQueryId() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + + bool executeStep() override; + +private: + String name; + MergeTreeData::MutableDataPartsVector parts; + const ProjectionDescription & projection; + size_t & block_num; + + ContextPtr context; + TableLockHolder * table_lock_holder; + MergeTreeDataMergerMutator * mutator; + MergeListEntry * merge_entry; + time_t time_of_merge; + + MergeTreeData::MutableDataPartPtr new_data_part; + ReservationSharedPtr space_reservation; + + LoggerPtr log; + + std::map level_parts; + size_t current_level = 0; + size_t next_level = 1; + + /// TODO(nikitamikhaylov): make this constant a setting + static constexpr size_t max_parts_to_merge_in_one_level = 10; +}; + +} diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 34e17e40a74..4be1b003573 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include #include #include @@ -47,12 +49,12 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } - /// PK columns are sorted and merged, ordinary columns are gathered using info from merge step static void extractMergingAndGatheringColumns( const NamesAndTypesList & storage_columns, const ExpressionActionsPtr & sorting_key_expr, const IndicesDescription & indexes, + const std::vector & projections, const MergeTreeData::MergingParams & merging_params, NamesAndTypesList & gathering_columns, Names & gathering_column_names, NamesAndTypesList & merging_columns, Names & merging_column_names) @@ -65,6 +67,12 @@ static void extractMergingAndGatheringColumns( std::copy(index_columns_vec.cbegin(), index_columns_vec.cend(), std::inserter(key_columns, key_columns.end())); } + for (const auto & projection : projections) + { + Names projection_columns_vec = projection->getRequiredColumns(); + std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(), + std::inserter(key_columns, key_columns.end())); + } /// Force sign column for Collapsing mode if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) @@ -203,10 +211,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() extendObjectColumns(global_ctx->storage_columns, object_columns, false); global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns)); + prepareProjectionsToMergeAndRebuild(); + extractMergingAndGatheringColumns( global_ctx->storage_columns, global_ctx->metadata_snapshot->getSortingKey().expression, global_ctx->metadata_snapshot->getSecondaryIndices(), + global_ctx->projections_to_rebuild, ctx->merging_params, global_ctx->gathering_columns, global_ctx->gathering_column_names, @@ -453,6 +464,65 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() } +void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Block & block) const +{ + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto projection_block = ctx->projection_squashes[i].add(projection.calculate(block, global_ctx->context)); + if (projection_block) + { + auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, projection_block, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + } + } +} + + +void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPartsMerge() const +{ + auto && [name, parts] = *ctx->projection_parts_iterator; + const auto & projection = global_ctx->metadata_snapshot->projections.get(name); + + ctx->merge_projection_parts_task_ptr = std::make_unique + ( + name, + std::move(parts), + projection, + ctx->projection_block_num, + global_ctx->context, + global_ctx->holder, + global_ctx->mutator, + global_ctx->merge_entry, + global_ctx->time_of_merge, + global_ctx->new_data_part, + global_ctx->space_reservation + ); +} + + +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() +{ + /// In case if there are no projections we didn't construct a task + if (!ctx->merge_projection_parts_task_ptr) + return false; + + if (ctx->merge_projection_parts_task_ptr->executeStep()) + return true; + + ++ctx->projection_parts_iterator; + + if (ctx->projection_parts_iterator == std::make_move_iterator(ctx->projection_parts.end())) + return false; + + constructTaskForProjectionPartsMerge(); + + return true; +} + bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { Block block; @@ -462,6 +532,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const_cast(*global_ctx->to).write(block); + calculateProjections(block); + UInt64 result_rows = 0; UInt64 result_bytes = 0; global_ctx->merged_pipeline.tryGetResultRowsAndBytes(result_rows, result_bytes); @@ -484,6 +556,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } + // finalize projections + calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + + ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); + if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) + constructTaskForProjectionPartsMerge(); + global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -732,24 +811,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c } - const auto & projections = global_ctx->metadata_snapshot->getProjections(); - - for (const auto & projection : projections) + for (const auto & projection : global_ctx->projections_to_merge) { - MergeTreeData::DataPartsVector projection_parts; - for (const auto & part : global_ctx->future_part->parts) - { - auto actual_projection_parts = part->getProjectionParts(); - auto it = actual_projection_parts.find(projection.name); - if (it != actual_projection_parts.end() && !it->second->is_broken) - projection_parts.push_back(it->second); - } - if (projection_parts.size() < global_ctx->future_part->parts.size()) - { - LOG_DEBUG(ctx->log, "Projection {} is not merged because some parts don't have it", projection.name); - continue; - } - + MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; LOG_DEBUG( ctx->log, "Selected {} projection_parts from {} to {}", @@ -759,7 +823,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c auto projection_future_part = std::make_shared(); projection_future_part->assign(std::move(projection_parts)); - projection_future_part->name = projection.name; + projection_future_part->name = projection->name; // TODO (ab): path in future_part is only for merge process introspection, which is not available for merges of projection parts. // Let's comment this out to avoid code inconsistency and add it back after we implement projection merge introspection. // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; @@ -767,16 +831,17 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; - if (projection.type == ProjectionDescription::Type::Aggregate) + if (projection->type == ProjectionDescription::Type::Aggregate) projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; ctx->tasks_for_projections.emplace_back(std::make_shared( projection_future_part, - projection.metadata, + projection->metadata, global_ctx->merge_entry, std::make_unique((*global_ctx->merge_entry)->table_id, projection_future_part, global_ctx->context), global_ctx->time_of_merge, global_ctx->context, + *global_ctx->holder, global_ctx->space_reservation, global_ctx->deduplicate, global_ctx->deduplicate_by_columns, @@ -1136,6 +1201,56 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->merging_executor = std::make_unique(global_ctx->merged_pipeline); } +void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const +{ + // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. + const bool merge_may_reduce_rows = + global_ctx->deduplicate || + ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + + const auto & projections = global_ctx->metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + if (merge_may_reduce_rows) + { + global_ctx->projections_to_rebuild.push_back(&projection); + continue; + } + + MergeTreeData::DataPartsVector projection_parts; + for (const auto & part : global_ctx->future_part->parts) + { + auto it = part->getProjectionParts().find(projection.name); + if (it != part->getProjectionParts().end()) + projection_parts.push_back(it->second); + } + if (projection_parts.size() == global_ctx->future_part->parts.size()) + { + global_ctx->projections_to_merge.push_back(&projection); + global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); + } + else if (projection_parts.empty()) + { + LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); + } + else + { + LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); + global_ctx->projections_to_rebuild.push_back(&projection); + } + } + + const auto & settings = global_ctx->context->getSettingsRef(); + + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + ctx->projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + } +} + MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const { diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index f6268886b14..e373b6b33c5 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -64,6 +65,7 @@ public: std::unique_ptr projection_merge_list_element_, time_t time_of_merge_, ContextPtr context_, + TableLockHolder & holder, ReservationSharedPtr space_reservation_, bool deduplicate_, Names deduplicate_by_columns_, @@ -88,6 +90,7 @@ public: = global_ctx->projection_merge_list_element ? global_ctx->projection_merge_list_element.get() : (*global_ctx->merge_entry)->ptr(); global_ctx->time_of_merge = std::move(time_of_merge_); global_ctx->context = std::move(context_); + global_ctx->holder = &holder; global_ctx->space_reservation = std::move(space_reservation_); global_ctx->deduplicate = std::move(deduplicate_); global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_); @@ -142,6 +145,7 @@ private: /// Proper initialization is responsibility of the author struct GlobalRuntimeContext : public IStageRuntimeContext { + TableLockHolder * holder; MergeList::Entry * merge_entry{nullptr}; /// If not null, use this instead of the global MergeList::Entry. This is for merging projections. std::unique_ptr projection_merge_list_element; @@ -173,6 +177,10 @@ private: MergeAlgorithm chosen_merge_algorithm{MergeAlgorithm::Undecided}; size_t gathering_column_names_size{0}; + std::vector projections_to_rebuild{}; + std::vector projections_to_merge{}; + std::map projections_to_merge_parts{}; + std::unique_ptr horizontal_stage_progress{nullptr}; std::unique_ptr column_progress{nullptr}; @@ -219,6 +227,14 @@ private: std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; + // For projections to rebuild + using ProjectionNameToItsBlocks = std::map; + ProjectionNameToItsBlocks projection_parts; + std::move_iterator projection_parts_iterator; + std::vector projection_squashes; + size_t projection_block_num = 0; + ExecutableTaskPtr merge_projection_parts_task_ptr; + size_t initial_reservation{0}; bool read_with_direct_io{false}; @@ -247,16 +263,21 @@ private: bool executeImpl(); /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable - using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; + using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; const ExecuteAndFinalizeHorizontalPartSubtasks subtasks { &ExecuteAndFinalizeHorizontalPart::prepare, - &ExecuteAndFinalizeHorizontalPart::executeImpl + &ExecuteAndFinalizeHorizontalPart::executeImpl, + &ExecuteAndFinalizeHorizontalPart::executeMergeProjections }; ExecuteAndFinalizeHorizontalPartSubtasks::const_iterator subtasks_iterator = subtasks.begin(); + void prepareProjectionsToMergeAndRebuild() const; + void calculateProjections(const Block & block) const; + void constructTaskForProjectionPartsMerge() const; + bool executeMergeProjections(); MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream(); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 53d49b51e8f..765c2c5e428 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -670,7 +670,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const StorageMetadataPtr & metadata_snapshot, MergeList::Entry * merge_entry, std::unique_ptr projection_merge_list_element, - TableLockHolder, + TableLockHolder & holder, time_t time_of_merge, ContextPtr context, ReservationSharedPtr space_reservation, @@ -690,6 +690,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( std::move(projection_merge_list_element), time_of_merge, context, + holder, space_reservation, deduplicate, deduplicate_by_columns, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 669ee040af3..fc910f401b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -159,7 +159,7 @@ public: const StorageMetadataPtr & metadata_snapshot, MergeListEntry * merge_entry, std::unique_ptr projection_merge_list_element, - TableLockHolder table_lock_holder, + TableLockHolder & table_lock_holder, time_t time_of_merge, ContextPtr context, ReservationSharedPtr space_reservation, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a971c4fda1c..aaf4e723494 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -1030,136 +1031,6 @@ struct MutationContext using MutationContextPtr = std::shared_ptr; -class MergeProjectionPartsTask : public IExecutableTask -{ -public: - - MergeProjectionPartsTask( - String name_, - MergeTreeData::MutableDataPartsVector && parts_, - const ProjectionDescription & projection_, - size_t & block_num_, - MutationContextPtr ctx_) - : name(std::move(name_)) - , parts(std::move(parts_)) - , projection(projection_) - , block_num(block_num_) - , ctx(ctx_) - , log(getLogger("MergeProjectionPartsTask")) - { - LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); - level_parts[current_level] = std::move(parts); - } - - void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - StorageID getStorageID() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - Priority getPriority() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - String getQueryId() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - - bool executeStep() override - { - auto & current_level_parts = level_parts[current_level]; - auto & next_level_parts = level_parts[next_level]; - - MergeTreeData::MutableDataPartsVector selected_parts; - while (selected_parts.size() < max_parts_to_merge_in_one_level && !current_level_parts.empty()) - { - selected_parts.push_back(std::move(current_level_parts.back())); - current_level_parts.pop_back(); - } - - if (selected_parts.empty()) - { - if (next_level_parts.empty()) - { - LOG_WARNING(log, "There is no projection parts merged"); - - /// Task is finished - return false; - } - current_level = next_level; - ++next_level; - } - else if (selected_parts.size() == 1) - { - if (next_level_parts.empty()) - { - LOG_DEBUG(log, "Merged a projection part in level {}", current_level); - selected_parts[0]->renameTo(projection.name + ".proj", true); - selected_parts[0]->setName(projection.name); - selected_parts[0]->is_temp = false; - ctx->new_data_part->addProjectionPart(name, std::move(selected_parts[0])); - - /// Task is finished - return false; - } - else - { - LOG_DEBUG(log, "Forwarded part {} in level {} to next level", selected_parts[0]->name, current_level); - next_level_parts.push_back(std::move(selected_parts[0])); - } - } - else if (selected_parts.size() > 1) - { - // Generate a unique part name - ++block_num; - auto projection_future_part = std::make_shared(); - MergeTreeData::DataPartsVector const_selected_parts( - std::make_move_iterator(selected_parts.begin()), std::make_move_iterator(selected_parts.end())); - projection_future_part->assign(std::move(const_selected_parts)); - projection_future_part->name = fmt::format("{}_{}", projection.name, ++block_num); - projection_future_part->part_info = {"all", 0, 0, 0}; - - MergeTreeData::MergingParams projection_merging_params; - projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; - if (projection.type == ProjectionDescription::Type::Aggregate) - projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; - - LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part->name); - auto tmp_part_merge_task = ctx->mutator->mergePartsToTemporaryPart( - projection_future_part, - projection.metadata, - ctx->mutate_entry, - std::make_unique((*ctx->mutate_entry)->table_id, projection_future_part, ctx->context), - *ctx->holder, - ctx->time_of_mutation, - ctx->context, - ctx->space_reservation, - false, // TODO Do we need deduplicate for projections - {}, - false, // no cleanup - projection_merging_params, - NO_TRANSACTION_PTR, - /* need_prefix */ true, - ctx->new_data_part.get(), - ".tmp_proj"); - - next_level_parts.push_back(executeHere(tmp_part_merge_task)); - next_level_parts.back()->is_temp = true; - } - - /// Need execute again - return true; - } - -private: - String name; - MergeTreeData::MutableDataPartsVector parts; - const ProjectionDescription & projection; - size_t & block_num; - MutationContextPtr ctx; - - LoggerPtr log; - - std::map level_parts; - size_t current_level = 0; - size_t next_level = 1; - - /// TODO(nikitamikhaylov): make this constant a setting - static constexpr size_t max_parts_to_merge_in_one_level = 10; -}; - - // This class is responsible for: // 1. get projection pipeline and a sink to write parts // 2. build an executor that can write block to the input stream (actually we can write through it to generate as many parts as possible) @@ -1356,7 +1227,13 @@ void PartMergerWriter::constructTaskForProjectionPartsMerge() std::move(parts), projection, block_num, - ctx + ctx->context, + ctx->holder, + ctx->mutator, + ctx->mutate_entry, + ctx->time_of_mutation, + ctx->new_data_part, + ctx->space_reservation ); } From 603a52caa06552c50e3b4d29989e177d2a0a3efa Mon Sep 17 00:00:00 2001 From: cangyin Date: Mon, 8 Apr 2024 12:34:59 +0800 Subject: [PATCH 02/41] Add tests --- .../02968_projection_merge.reference | 28 +++++ .../0_stateless/02968_projection_merge.sql | 112 ++++++++++++++++++ 2 files changed, 140 insertions(+) create mode 100644 tests/queries/0_stateless/02968_projection_merge.reference create mode 100644 tests/queries/0_stateless/02968_projection_merge.sql diff --git a/tests/queries/0_stateless/02968_projection_merge.reference b/tests/queries/0_stateless/02968_projection_merge.reference new file mode 100644 index 00000000000..40cb572c95a --- /dev/null +++ b/tests/queries/0_stateless/02968_projection_merge.reference @@ -0,0 +1,28 @@ +ReplacingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +CollapsingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +VersionedCollapsingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +DEDUPLICATE ON MergeTree +0 1 +1 1 +2 1 +0 1 +1 1 +2 1 diff --git a/tests/queries/0_stateless/02968_projection_merge.sql b/tests/queries/0_stateless/02968_projection_merge.sql new file mode 100644 index 00000000000..07d40e30c2f --- /dev/null +++ b/tests/queries/0_stateless/02968_projection_merge.sql @@ -0,0 +1,112 @@ +SELECT 'ReplacingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = ReplacingMergeTree +ORDER BY type; + +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); +INSERT INTO tp SELECT number%3, 2 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + + +SELECT 'CollapsingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + `sign` Int8, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = CollapsingMergeTree(sign) +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 1, -1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2, 1 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +-- Actually we don't need to test all 3 engines Replacing/Collapsing/VersionedCollapsing, +-- Because they share the same logic of 'reduce number of rows during merges' +SELECT 'VersionedCollapsingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + `sign` Int8, + `version` UInt8, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = VersionedCollapsingMergeTree(sign,version) +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1, -1, 0 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2, 1, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 1, 1, 0 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +SELECT 'DEDUPLICATE ON MergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = MergeTree +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL DEDUPLICATE BY type; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + From 8c2a371eaa8e51c0d382a13f4413ab9a2796e02b Mon Sep 17 00:00:00 2001 From: cangyin Date: Mon, 8 Apr 2024 18:52:08 +0800 Subject: [PATCH 03/41] no readability-make-member-function-const --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4be1b003573..66629163ebb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -504,7 +504,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT { /// In case if there are no projections we didn't construct a task if (!ctx->merge_projection_parts_task_ptr) From 5ecb5da648afb4d2d62aa7edee55af4ae89e7459 Mon Sep 17 00:00:00 2001 From: cangyin Date: Wed, 10 Apr 2024 19:30:28 +0000 Subject: [PATCH 04/41] Also rebuild for OPTIMIZE CLEANUP --- src/Storages/MergeTree/MergeTask.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 66629163ebb..72a75e4a32e 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1205,6 +1205,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe { // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. const bool merge_may_reduce_rows = + global_ctx->cleanup || global_ctx->deduplicate || ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || From 6f6056477785842c047ab0faa4c089a514625f06 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 7 Aug 2024 02:28:13 +0000 Subject: [PATCH 05/41] fix build --- src/Storages/MergeTree/MergeTask.cpp | 12 +++++++----- src/Storages/MergeTree/MergeTask.h | 4 ++-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 2e7cc03f5ed..e50a66c91ec 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -501,11 +501,13 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) { const auto & projection = *global_ctx->projections_to_rebuild[i]; - auto projection_block = ctx->projection_squashes[i].add(projection.calculate(block, global_ctx->context)); - if (projection_block) + Block block_to_squash = projection.calculate(block, global_ctx->context); + auto chunk = ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); + if (chunk) { + auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *global_ctx->data, ctx->log, projection_block, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); ctx->projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); @@ -1313,9 +1315,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe const auto & settings = global_ctx->context->getSettingsRef(); - for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + for (auto projection : global_ctx->projections_to_rebuild) { - ctx->projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + ctx->projection_squashes.emplace_back(projection->sample_block, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 0424b698f8a..f60a32252b2 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -231,7 +231,7 @@ private: using ProjectionNameToItsBlocks = std::map; ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squashes; + std::vector projection_squashes; size_t projection_block_num = 0; ExecutableTaskPtr merge_projection_parts_task_ptr; From 22dad244e75137580e08aa2cc83538e5ffd950c8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 8 Aug 2024 01:50:49 +0000 Subject: [PATCH 06/41] fix whitespace --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 49e9121b65a..4133e47d4bc 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -887,7 +887,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c ctx->projections_iterator = ctx->tasks_for_projections.begin(); return false; } - + for (const auto & projection : global_ctx->projections_to_merge) { MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; From a837df164c48f0b8041b122adc8fa80a148629f8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 02:37:42 +0000 Subject: [PATCH 07/41] fix squash related and projection collection --- src/Storages/MergeTree/MergeTask.cpp | 47 ++++++++++++------- .../0_stateless/02968_projection_merge.sql | 12 +++-- 2 files changed, 38 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4133e47d4bc..3b117168e33 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -502,10 +502,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo { const auto & projection = *global_ctx->projections_to_rebuild[i]; Block block_to_squash = projection.calculate(block, global_ctx->context); - auto chunk = ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); - if (chunk) + ctx->projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + auto squashed_chunk = Squashing::squash(ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + if (squashed_chunk) { - auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(chunk.detachColumns()); + auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); @@ -590,8 +591,23 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } - // finalize projections - calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + /// finalize projections + // calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto & projection_squash_plan = ctx->projection_squashes[i]; + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) + { + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + temp_part.finalize(); + temp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); + } + } ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) @@ -878,16 +894,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c ReadableSize(global_ctx->merge_list_element_ptr->bytes_read_uncompressed / elapsed_seconds)); } - const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; - /// Under throw mode, we still choose to drop projections due to backward compatibility since some - /// users might have projections before this change. - if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary - && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) - { - ctx->projections_iterator = ctx->tasks_for_projections.begin(); - return false; - } - for (const auto & projection : global_ctx->projections_to_merge) { MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; @@ -1281,6 +1287,13 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const { + const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; + /// Under throw mode, we still choose to drop projections due to backward compatibility since some + /// users might have projections before this change. + if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary + && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) + return; + // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. const bool merge_may_reduce_rows = global_ctx->cleanup || @@ -1324,9 +1337,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe const auto & settings = global_ctx->context->getSettingsRef(); - for (auto projection : global_ctx->projections_to_rebuild) + for (const auto * projection : global_ctx->projections_to_rebuild) { - ctx->projection_squashes.emplace_back(projection->sample_block, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } } diff --git a/tests/queries/0_stateless/02968_projection_merge.sql b/tests/queries/0_stateless/02968_projection_merge.sql index 07d40e30c2f..03488042a4e 100644 --- a/tests/queries/0_stateless/02968_projection_merge.sql +++ b/tests/queries/0_stateless/02968_projection_merge.sql @@ -11,7 +11,8 @@ CREATE TABLE tp ) ) ENGINE = ReplacingMergeTree -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number%3, 1 FROM numbers(3); INSERT INTO tp SELECT number%3, 2 FROM numbers(3); @@ -39,7 +40,8 @@ CREATE TABLE tp ) ) ENGINE = CollapsingMergeTree(sign) -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1, 1 FROM numbers(3); INSERT INTO tp SELECT number % 3, 1, -1 FROM numbers(3); @@ -70,7 +72,8 @@ CREATE TABLE tp ) ) ENGINE = VersionedCollapsingMergeTree(sign,version) -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1, -1, 0 FROM numbers(3); INSERT INTO tp SELECT number % 3, 2, 1, 1 FROM numbers(3); @@ -97,7 +100,8 @@ CREATE TABLE tp ) ) ENGINE = MergeTree -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1 FROM numbers(3); INSERT INTO tp SELECT number % 3, 2 FROM numbers(3); From 3cfb921befa895e445e8d7b98e639015e1e41aa0 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 18:41:53 +0000 Subject: [PATCH 08/41] Fix using schema_inference_make_columns_nullable=0 --- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 4 +++ src/Processors/Formats/ISchemaReader.cpp | 2 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 7 +++-- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 9 ++++--- .../Formats/Impl/ArrowColumnToCHColumn.h | 3 ++- .../Impl/NativeORCBlockInputFormat.cpp | 2 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 7 +++-- .../Formats/Impl/ParquetBlockInputFormat.cpp | 7 +++-- .../03036_parquet_arrow_nullable.reference | 26 +++++++++++++++++++ .../03036_parquet_arrow_nullable.sh | 7 +++++ 13 files changed, 65 insertions(+), 15 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0808e8eb49f..bc9c6daab1b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1116,7 +1116,7 @@ class IColumn; M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \ M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ - M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ + M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHosue will use information about nullability from the data..", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a78836ff63c..8d8257b9abc 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -255,7 +255,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_bytes_to_read_for_schema_inference; format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference; format_settings.schema_inference_hints = settings.schema_inference_hints; - format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable; + format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable.valueOr(2); format_settings.mysql_dump.table_name = settings.input_format_mysql_dump_table_name; format_settings.mysql_dump.map_column_names = settings.input_format_mysql_dump_map_column_names; format_settings.sql_insert.max_batch_size = settings.output_format_sql_insert_max_batch_size; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index f0359218775..479b1a89adf 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -75,7 +75,7 @@ struct FormatSettings Raw }; - bool schema_inference_make_columns_nullable = true; + UInt64 schema_inference_make_columns_nullable = true; DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 3c374ada9e6..c04682e8765 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1262,7 +1262,11 @@ namespace if (checkCharCaseInsensitive('n', buf)) { if (checkStringCaseInsensitive("ull", buf)) + { + if (settings.schema_inference_make_columns_nullable == 0) + return std::make_shared(); return makeNullable(std::make_shared()); + } else if (checkStringCaseInsensitive("an", buf)) return std::make_shared(); } diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 45523700a5d..569d4bb39e7 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -54,7 +54,7 @@ void checkFinalInferredType( type = default_type; } - if (settings.schema_inference_make_columns_nullable) + if (settings.schema_inference_make_columns_nullable == 1) type = makeNullableRecursively(type); /// In case when data for some column could contain nulls and regular values, /// resulting inferred type is Nullable. diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 72a93002669..cf079e52db0 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -204,8 +204,11 @@ NamesAndTypesList ArrowSchemaReader::readSchema() schema = file_reader->schema(); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, stream ? "ArrowStream" : "Arrow", format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + stream ? "ArrowStream" : "Arrow", + format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index ed91913de4d..bcc8bfecdc6 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -712,6 +712,7 @@ struct ReadColumnFromArrowColumnSettings FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior; bool allow_arrow_null_type; bool skip_columns_with_unsupported_types; + bool allow_inferring_nullable_columns; }; static ColumnWithTypeAndName readColumnFromArrowColumn( @@ -1085,7 +1086,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( bool is_map_nested_column, const ReadColumnFromArrowColumnSettings & settings) { - bool read_as_nullable_column = arrow_column->null_count() || is_nullable_column || (type_hint && type_hint->isNullable()); + bool read_as_nullable_column = (arrow_column->null_count() || is_nullable_column || (type_hint && type_hint->isNullable())) && settings.allow_inferring_nullable_columns; if (read_as_nullable_column && arrow_column->type()->id() != arrow::Type::LIST && arrow_column->type()->id() != arrow::Type::LARGE_LIST && @@ -1149,14 +1150,16 @@ static std::shared_ptr createArrowColumn(const std::shared_ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types) + bool skip_columns_with_unsupported_types, + bool allow_inferring_nullable_columns) { ReadColumnFromArrowColumnSettings settings { .format_name = format_name, .date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore, .allow_arrow_null_type = false, - .skip_columns_with_unsupported_types = skip_columns_with_unsupported_types + .skip_columns_with_unsupported_types = skip_columns_with_unsupported_types, + .allow_inferring_nullable_columns = allow_inferring_nullable_columns, }; ColumnsWithTypeAndName sample_columns; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 27e9afdf763..8521cd2f410 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -34,7 +34,8 @@ public: static Block arrowSchemaToCHHeader( const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types = false); + bool skip_columns_with_unsupported_types = false, + bool allow_inferring_nullable_columns = true); struct DictionaryInfo { diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 58bec8120f1..b0fd6789d1a 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1002,7 +1002,7 @@ NamesAndTypesList NativeORCSchemaReader::readSchema() header.insert(ColumnWithTypeAndName{type, name}); } - if (format_settings.schema_inference_make_columns_nullable) + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index a3c218fa26e..2266c0b488c 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -160,8 +160,11 @@ NamesAndTypesList ORCSchemaReader::readSchema() { initializeIfNeeded(); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + "ORC", + format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index bc5e8292192..b116070b8df 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -810,8 +810,11 @@ NamesAndTypesList ParquetSchemaReader::readSchema() THROW_ARROW_NOT_OK(parquet::arrow::FromParquetSchema(metadata->schema(), &schema)); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, "Parquet", format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + "Parquet", + format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference index 985f8192f26..d15f0d8365d 100644 --- a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference @@ -1,40 +1,66 @@ Parquet a UInt64 a_nullable Nullable(UInt64) +a UInt64 +a_nullable UInt64 Arrow a UInt64 a_nullable Nullable(UInt64) +a UInt64 +a_nullable UInt64 Parquet b Array(UInt64) b_nullable Array(Nullable(UInt64)) +b Array(UInt64) +b_nullable Array(UInt64) Arrow b Array(Nullable(UInt64)) b_nullable Array(Nullable(UInt64)) +b Array(UInt64) +b_nullable Array(UInt64) Parquet c Tuple(\n a UInt64,\n b String) c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a UInt64,\n b String) Arrow c Tuple(\n a UInt64,\n b String) c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a UInt64,\n b String) Parquet d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String))) Arrow d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String))) Parquet e Map(UInt64, String) e_nullable Map(UInt64, Nullable(String)) +e Map(UInt64, String) +e_nullable Map(UInt64, String) Arrow e Map(UInt64, Nullable(String)) e_nullable Map(UInt64, Nullable(String)) +e Map(UInt64, String) +e_nullable Map(UInt64, String) Parquet f Map(UInt64, Map(UInt64, String)) f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +f Map(UInt64, Map(UInt64, String)) +f_nullables Map(UInt64, Map(UInt64, String)) Arrow f Map(UInt64, Map(UInt64, Nullable(String))) f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +f Map(UInt64, Map(UInt64, String)) +f_nullables Map(UInt64, Map(UInt64, String)) Parquet g String g_nullable Nullable(String) +g String +g_nullable String Arrow g LowCardinality(String) g_nullable LowCardinality(String) +g LowCardinality(String) +g_nullable LowCardinality(String) diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh index bdd641e2b94..379756f78f3 100755 --- a/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh @@ -14,6 +14,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, a_nullable Nullable(UInt64)', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -21,6 +22,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('b Array(UInt64), b_nullable Array(Nullable(UInt64))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -28,6 +30,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('c Tuple(a UInt64, b String), c_nullable Tuple(a Nullable(UInt64), b Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -35,6 +38,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('d Tuple(a UInt64, b Tuple(a UInt64, b String), d_nullable Tuple(a UInt64, b Tuple(a Nullable(UInt64), b Nullable(String))))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -42,6 +46,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('e Map(UInt64, String), e_nullable Map(UInt64, Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -49,6 +54,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('f Map(UInt64, Map(UInt64, String)), f_nullables Map(UInt64, Map(UInt64, Nullable(String)))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -56,6 +62,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('g LowCardinality(String), g_nullable LowCardinality(Nullable(String))', 42) limit 10 settings output_format_arrow_low_cardinality_as_dictionary=1, allow_suspicious_low_cardinality_types=1 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done From 70708fd5dcf633d4d3147240195554587f4fb14f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 19:19:02 +0000 Subject: [PATCH 09/41] Update docs, make better --- docs/en/interfaces/schema-inference.md | 7 ++----- docs/en/operations/settings/settings-formats.md | 4 ++-- src/Processors/Formats/ISchemaReader.cpp | 5 ----- 3 files changed, 4 insertions(+), 12 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 05fae994cbe..5b3cd179e21 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1385,7 +1385,7 @@ DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : nul #### schema_inference_make_columns_nullable Controls making inferred types `Nullable` in schema inference for formats without information about nullability. -If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if `input_format_null_as_default` is disabled and the column contains `NULL` in a sample that is parsed during schema inference. +If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will never be `Nullable`, if set to `auto`, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference or file metadata contains information about column nullability. Enabled by default. @@ -1408,15 +1408,13 @@ DESC format(JSONEachRow, $$ └─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` ```sql -SET schema_inference_make_columns_nullable = 0; -SET input_format_null_as_default = 0; +SET schema_inference_make_columns_nullable = 'auto'; DESC format(JSONEachRow, $$ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} $$) ``` ```response - ┌─name────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ │ id │ Int64 │ │ │ │ │ │ │ age │ Int64 │ │ │ │ │ │ @@ -1428,7 +1426,6 @@ DESC format(JSONEachRow, $$ ```sql SET schema_inference_make_columns_nullable = 0; -SET input_format_null_as_default = 1; DESC format(JSONEachRow, $$ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index f8b40cd81ac..57812ef0e03 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -171,8 +171,8 @@ If the `schema_inference_hints` is not formated properly, or if there is a typo ## schema_inference_make_columns_nullable {#schema_inference_make_columns_nullable} -Controls making inferred types `Nullable` in schema inference for formats without information about nullability. -If the setting is enabled, the inferred type will be `Nullable` only if column contains `NULL` in a sample that is parsed during schema inference. +Controls making inferred types `Nullable` in schema inference. +If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will never be `Nullable`, if set to `auto`, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference or file metadata contains information about column nullability. Default value: `true`. diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 569d4bb39e7..e002e64b7e5 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -56,11 +56,6 @@ void checkFinalInferredType( if (settings.schema_inference_make_columns_nullable == 1) type = makeNullableRecursively(type); - /// In case when data for some column could contain nulls and regular values, - /// resulting inferred type is Nullable. - /// If input_format_null_as_default is enabled, we should remove Nullable type. - else if (settings.null_as_default) - type = removeNullable(type); } void ISchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) From 6bed26a52764558ba2b52752bdfd126fd943c616 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 14 Aug 2024 02:19:34 +0000 Subject: [PATCH 10/41] tidy --- src/Storages/MergeTree/MergeTask.cpp | 171 ++++++++++++++------------- src/Storages/MergeTree/MergeTask.h | 3 +- 2 files changed, 90 insertions(+), 84 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3b117168e33..56e17ac1884 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -145,7 +145,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu } } - for (const auto & projection : global_ctx->projections_to_rebuild) + for (const auto * projection : global_ctx->projections_to_rebuild) { Names projection_columns_vec = projection->getRequiredColumns(); std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(), @@ -496,17 +496,76 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() } +void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const +{ + const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; + /// Under throw mode, we still choose to drop projections due to backward compatibility since some + /// users might have projections before this change. + if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary + && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) + return; + + /// These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. + const bool merge_may_reduce_rows = + global_ctx->cleanup || + global_ctx->deduplicate || + ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + + const auto & projections = global_ctx->metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + if (merge_may_reduce_rows) + { + global_ctx->projections_to_rebuild.push_back(&projection); + continue; + } + + MergeTreeData::DataPartsVector projection_parts; + for (const auto & part : global_ctx->future_part->parts) + { + auto it = part->getProjectionParts().find(projection.name); + if (it != part->getProjectionParts().end()) + projection_parts.push_back(it->second); + } + if (projection_parts.size() == global_ctx->future_part->parts.size()) + { + global_ctx->projections_to_merge.push_back(&projection); + global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); + } + else if (projection_parts.empty()) + { + LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); + } + else + { + LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); + global_ctx->projections_to_rebuild.push_back(&projection); + } + } + + const auto & settings = global_ctx->context->getSettingsRef(); + + for (const auto * projection : global_ctx->projections_to_rebuild) + ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), + settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); +} + + void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Block & block) const { for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) { const auto & projection = *global_ctx->projections_to_rebuild[i]; Block block_to_squash = projection.calculate(block, global_ctx->context); - ctx->projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - auto squashed_chunk = Squashing::squash(ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + auto & projection_squash_plan = ctx->projection_squashes[i]; + projection_squash_plan.setHeader(block_to_squash.cloneEmpty()); + Chunk squashed_chunk = Squashing::squash(projection_squash_plan.add({block_to_squash.getColumns(), block_to_squash.rows()})); if (squashed_chunk) { - auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); @@ -517,6 +576,30 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo } +void MergeTask::ExecuteAndFinalizeHorizontalPart::finalizeProjections() const +{ + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto & projection_squash_plan = ctx->projection_squashes[i]; + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) + { + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + temp_part.finalize(); + temp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); + } + } + + ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); + if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) + constructTaskForProjectionPartsMerge(); +} + + void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPartsMerge() const { auto && [name, parts] = *ctx->projection_parts_iterator; @@ -591,27 +674,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } - /// finalize projections - // calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); - for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) - { - const auto & projection = *global_ctx->projections_to_rebuild[i]; - auto & projection_squash_plan = ctx->projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); - if (squashed_chunk) - { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); - temp_part.finalize(); - temp_part.part->getDataPartStorage().commitTransaction(); - ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); - } - } - - ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); - if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) - constructTaskForProjectionPartsMerge(); + finalizeProjections(); global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -1285,64 +1348,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->merging_executor = std::make_unique(global_ctx->merged_pipeline); } -void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const -{ - const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; - /// Under throw mode, we still choose to drop projections due to backward compatibility since some - /// users might have projections before this change. - if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary - && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) - return; - - // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. - const bool merge_may_reduce_rows = - global_ctx->cleanup || - global_ctx->deduplicate || - ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || - ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || - ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; - - const auto & projections = global_ctx->metadata_snapshot->getProjections(); - - for (const auto & projection : projections) - { - if (merge_may_reduce_rows) - { - global_ctx->projections_to_rebuild.push_back(&projection); - continue; - } - - MergeTreeData::DataPartsVector projection_parts; - for (const auto & part : global_ctx->future_part->parts) - { - auto it = part->getProjectionParts().find(projection.name); - if (it != part->getProjectionParts().end()) - projection_parts.push_back(it->second); - } - if (projection_parts.size() == global_ctx->future_part->parts.size()) - { - global_ctx->projections_to_merge.push_back(&projection); - global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); - } - else if (projection_parts.empty()) - { - LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); - } - else - { - LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); - global_ctx->projections_to_rebuild.push_back(&projection); - } - } - - const auto & settings = global_ctx->context->getSettingsRef(); - - for (const auto * projection : global_ctx->projections_to_rebuild) - { - ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); - } -} - MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const { diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index f60a32252b2..d9cc7c1dbad 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -227,7 +227,7 @@ private: std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; - // For projections to rebuild + /// For projections to rebuild using ProjectionNameToItsBlocks = std::map; ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; @@ -275,6 +275,7 @@ private: void prepareProjectionsToMergeAndRebuild() const; void calculateProjections(const Block & block) const; + void finalizeProjections() const; void constructTaskForProjectionPartsMerge() const; bool executeMergeProjections(); From e2feaefcaf0e88f86f303c068edcbdacaeb67252 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:13:49 +0200 Subject: [PATCH 11/41] Update src/Core/Settings.h Co-authored-by: Alexey Katsman --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bc9c6daab1b..2417ddd39e8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1116,7 +1116,7 @@ class IColumn; M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \ M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ - M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHosue will use information about nullability from the data..", 0) \ + M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHouse will use information about nullability from the data.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ From ccb7ecb9a22ddeabe93a5b907e3ad688b04966b4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:13:57 +0200 Subject: [PATCH 12/41] Update src/Formats/FormatSettings.h Co-authored-by: Alexey Katsman --- src/Formats/FormatSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 479b1a89adf..81b34ff0c55 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -75,7 +75,7 @@ struct FormatSettings Raw }; - UInt64 schema_inference_make_columns_nullable = true; + UInt64 schema_inference_make_columns_nullable = 1; DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; From 92a9b29b45c254e670fe9f67114b5af890bfb5cb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 15 Aug 2024 22:25:21 +0800 Subject: [PATCH 13/41] devirtualize format reader --- .../Formats/Impl/BinaryRowInputFormat.cpp | 4 +- .../Formats/Impl/BinaryRowInputFormat.h | 7 ++- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- .../Formats/Impl/CSVRowInputFormat.h | 7 ++- .../Impl/CustomSeparatedRowInputFormat.h | 3 +- .../Impl/JSONCompactEachRowRowInputFormat.h | 4 +- .../Impl/JSONCompactRowInputFormat.cpp | 2 +- .../Formats/Impl/JSONCompactRowInputFormat.h | 4 +- .../Formats/Impl/TabSeparatedRowInputFormat.h | 4 +- .../RowInputFormatWithNamesAndTypes.cpp | 61 +++++++++++++------ .../Formats/RowInputFormatWithNamesAndTypes.h | 5 +- 11 files changed, 67 insertions(+), 36 deletions(-) diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index c5336f3bcc7..b549f2de975 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes } template -BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes( +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, IRowInputFormat::Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes>( header, in_, params_, diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 6f2042d1315..6a4ca8f6418 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -10,13 +10,16 @@ namespace DB class ReadBuffer; +template +class BinaryFormatReader; + /** A stream for inputting data in a binary line-by-line format. */ template -class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes +class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes> { public: - BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); + BinaryRowInputFormat(ReadBuffer & in_, const Block & header, IRowInputFormat::Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); String getName() const override { return "BinaryRowInputFormat"; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index b7f84748f61..cf58a4057c8 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -61,7 +61,7 @@ CSVRowInputFormat::CSVRowInputFormat( bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_) + std::unique_ptr format_reader_) : RowInputFormatWithNamesAndTypes( header_, *in_, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index fe4d4e3be08..86af5028438 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -13,10 +12,12 @@ namespace DB { +class CSVFormatReader; + /** A stream for inputting data in csv format. * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. */ -class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes +class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes { public: /** with_names - in the first line the header with column names @@ -32,7 +33,7 @@ public: protected: CSVRowInputFormat(const Block & header_, std::shared_ptr in_, const Params & params_, - bool with_names_, bool with_types_, const FormatSettings & format_settings_, std::unique_ptr format_reader_); + bool with_names_, bool with_types_, const FormatSettings & format_settings_, std::unique_ptr format_reader_); CSVRowInputFormat(const Block & header_, std::shared_ptr in_buf_, const Params & params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 58f78e5af42..b1d35947ba8 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -9,7 +9,8 @@ namespace DB { -class CustomSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes +class CustomSeparatedFormatReader; +class CustomSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: CustomSeparatedRowInputFormat( diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index ebeb939e7fa..50589329073 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -11,7 +11,7 @@ namespace DB { class ReadBuffer; - +class JSONCompactEachRowFormatReader; /** A stream for reading data in a bunch of formats: * - JSONCompactEachRow @@ -20,7 +20,7 @@ class ReadBuffer; * - JSONCompactStringsEachRowWithNamesAndTypes * */ -class JSONCompactEachRowRowInputFormat final : public RowInputFormatWithNamesAndTypes +class JSONCompactEachRowRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: JSONCompactEachRowRowInputFormat( diff --git a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp index 63066fc8220..63ced05dd3a 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes JSONCompactRowInputFormat::JSONCompactRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes( + : RowInputFormatWithNamesAndTypes( header_, in_, params_, false, false, false, format_settings_, std::make_unique(in_, format_settings_)) { } diff --git a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h index 3a93e7149b0..eb70f6ec2a3 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h @@ -5,8 +5,8 @@ namespace DB { - -class JSONCompactRowInputFormat final : public RowInputFormatWithNamesAndTypes +class JSONCompactFormatReader; +class JSONCompactRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: JSONCompactRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 32abd532a52..3c6efe9ac4c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -10,9 +10,11 @@ namespace DB { +class TabSeparatedFormatReader; + /** A stream to input data in tsv format. */ -class TabSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes +class TabSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: /** with_names - the first line is the header with the names of the columns diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index ae30d741c2f..5701b80ecc2 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -1,14 +1,20 @@ -#include -#include -#include -#include #include #include -#include -#include -#include -#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -44,7 +50,8 @@ namespace } } -RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( +template +RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( const Block & header_, ReadBuffer & in_, const Params & params_, @@ -52,7 +59,7 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_, + std::unique_ptr format_reader_, bool try_detect_header_) : RowInputFormatWithDiagnosticInfo(header_, in_, params_) , format_settings(format_settings_) @@ -66,7 +73,8 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( column_indexes_by_names = getPort().getHeader().getNamesToIndexesMap(); } -void RowInputFormatWithNamesAndTypes::readPrefix() +template +void RowInputFormatWithNamesAndTypes::readPrefix() { /// Search and remove BOM only in textual formats (CSV, TSV etc), not in binary ones (RowBinary*). /// Also, we assume that column name or type cannot contain BOM, so, if format has header, @@ -138,7 +146,8 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } } -void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & column_names_out, std::vector & type_names_out) +template +void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & column_names_out, std::vector & type_names_out) { auto & read_buf = getReadBuffer(); PeekableReadBuffer * peekable_buf = dynamic_cast(&read_buf); @@ -201,7 +210,8 @@ void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & colu peekable_buf->dropCheckpoint(); } -bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) +template +bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) { if (unlikely(end_of_stream)) return false; @@ -280,7 +290,8 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE return true; } -size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) +template +size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) { if (unlikely(end_of_stream)) return 0; @@ -304,7 +315,8 @@ size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) return num_rows; } -void RowInputFormatWithNamesAndTypes::resetParser() +template +void RowInputFormatWithNamesAndTypes::resetParser() { RowInputFormatWithDiagnosticInfo::resetParser(); column_mapping->column_indexes_for_input_fields.clear(); @@ -313,7 +325,8 @@ void RowInputFormatWithNamesAndTypes::resetParser() end_of_stream = false; } -void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) +template +void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) { const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; if (index) @@ -328,7 +341,8 @@ void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & ty } } -bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +template +bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) { if (in->eof()) { @@ -374,12 +388,14 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu return format_reader->parseRowEndWithDiagnosticInfo(out); } -bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos) +template +bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos) { return format_reader->isGarbageAfterField(index, pos); } -void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) +template +void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) { format_reader->setReadBuffer(in_); IInputFormat::setReadBuffer(in_); @@ -582,5 +598,12 @@ void FormatWithNamesAndTypesSchemaReader::transformTypesIfNeeded(DB::DataTypePtr transformInferredTypesIfNeeded(type, new_type, format_settings); } +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes>; +template class RowInputFormatWithNamesAndTypes>; } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index b7d9507151e..cd836cb00dc 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -26,6 +26,7 @@ class FormatWithNamesAndTypesReader; /// will be compared types from header. /// It's important that firstly this class reads/skips names and only /// then reads/skips types. So you can this invariant. +template class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo { protected: @@ -41,7 +42,7 @@ protected: bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_, + std::unique_ptr format_reader_, bool try_detect_header_ = false); void resetParser() override; @@ -70,7 +71,7 @@ private: bool is_header_detected = false; protected: - std::unique_ptr format_reader; + std::unique_ptr format_reader; Block::NameMap column_indexes_by_names; }; From 370b6bdc7b6d97f0e697e99ccd06a25e97651406 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:38:30 +0000 Subject: [PATCH 14/41] Update tests --- tests/queries/0_stateless/02497_schema_inference_nulls.sql | 4 ++-- .../0_stateless/02784_schema_inference_null_as_default.sql | 4 ++-- ..._max_bytes_to_read_for_schema_inference_in_cache.reference | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02497_schema_inference_nulls.sql b/tests/queries/0_stateless/02497_schema_inference_nulls.sql index b78b5709dbb..5670b031e8b 100644 --- a/tests/queries/0_stateless/02497_schema_inference_nulls.sql +++ b/tests/queries/0_stateless/02497_schema_inference_nulls.sql @@ -40,7 +40,7 @@ desc format(JSONCompactEachRow, '[[[], [null], [1, 2, 3]]]'); desc format(JSONCompactEachRow, '[[{"a" : null}, {"b" : 1}]]'); desc format(JSONCompactEachRow, '[[["2020-01-01", null, "1234"], ["abcd"]]]'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(JSONCompactEachRow, '[[1, 2]]'); desc format(JSONCompactEachRow, '[[null, 1]]'); desc format(JSONCompactEachRow, '[[1, 2]], [[3]]'); @@ -59,7 +59,7 @@ desc format(CSV, '"[[], [null], [1, 2, 3]]"'); desc format(CSV, '"[{\'a\' : null}, {\'b\' : 1}]"'); desc format(CSV, '"[[\'2020-01-01\', null, \'1234\'], [\'abcd\']]"'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(CSV, '"[1,2]"'); desc format(CSV, '"[NULL, 1]"'); desc format(CSV, '"[1, 2]"\n"[3]"'); diff --git a/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql index 9c9f99d8283..571e3ab4f25 100644 --- a/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql +++ b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql @@ -1,7 +1,7 @@ desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; -desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; -select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; +desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable='auto', input_format_null_as_default=0; +select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable='auto', input_format_null_as_default=0; desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference index cd109daac52..3b9d88edc19 100644 --- a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference @@ -1,2 +1,2 @@ x Nullable(Int64) -schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false +schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=1, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false From 5fe46af4221a36ecb4566ca7bfad314d732f1de2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:12:51 +0200 Subject: [PATCH 15/41] Update 02497_schema_inference_nulls.sql --- tests/queries/0_stateless/02497_schema_inference_nulls.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_schema_inference_nulls.sql b/tests/queries/0_stateless/02497_schema_inference_nulls.sql index 5670b031e8b..d62fc76d9b9 100644 --- a/tests/queries/0_stateless/02497_schema_inference_nulls.sql +++ b/tests/queries/0_stateless/02497_schema_inference_nulls.sql @@ -18,7 +18,7 @@ desc format(JSONEachRow, '{"x" : [[], [null], [1, 2, 3]]}'); desc format(JSONEachRow, '{"x" : [{"a" : null}, {"b" : 1}]}'); desc format(JSONEachRow, '{"x" : [["2020-01-01", null, "1234"], ["abcd"]]}'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(JSONEachRow, '{"x" : [1, 2]}'); desc format(JSONEachRow, '{"x" : [null, 1]}'); desc format(JSONEachRow, '{"x" : [1, 2]}, {"x" : [3]}'); From 527774d138fd2a8b077800c0897417cedb2113be Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 19 Aug 2024 15:26:17 +0000 Subject: [PATCH 16/41] use new option name --- .../integration/test_broken_projections/test.py | 2 +- .../0_stateless/02968_projection_merge.sql | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 578ff42369c..4ea018d3379 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -191,7 +191,7 @@ def optimize(node, table, final, no_wait): if final: query += " FINAL" if no_wait: - query += " SETTINGS alter_sync=0" + query += " SETTINGS alter_sync=2" node.query(query) diff --git a/tests/queries/0_stateless/02968_projection_merge.sql b/tests/queries/0_stateless/02968_projection_merge.sql index 03488042a4e..3e047d2cf69 100644 --- a/tests/queries/0_stateless/02968_projection_merge.sql +++ b/tests/queries/0_stateless/02968_projection_merge.sql @@ -20,10 +20,10 @@ INSERT INTO tp SELECT number%3, 2 FROM numbers(3); OPTIMIZE TABLE tp FINAL; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; +SETTINGS optimize_use_projections = 0, force_optimize_projection = 0; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +SETTINGS optimize_use_projections = 1, force_optimize_projection = 1; SELECT 'CollapsingMergeTree'; @@ -50,10 +50,10 @@ INSERT INTO tp SELECT number % 3, 2, 1 FROM numbers(3); OPTIMIZE TABLE tp FINAL; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; +SETTINGS optimize_use_projections = 0, force_optimize_projection = 0; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +SETTINGS optimize_use_projections = 1, force_optimize_projection = 1; -- Actually we don't need to test all 3 engines Replacing/Collapsing/VersionedCollapsing, -- Because they share the same logic of 'reduce number of rows during merges' @@ -82,10 +82,10 @@ INSERT INTO tp SELECT number % 3, 1, 1, 0 FROM numbers(3); OPTIMIZE TABLE tp FINAL; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; +SETTINGS optimize_use_projections = 0, force_optimize_projection = 0; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +SETTINGS optimize_use_projections = 1, force_optimize_projection = 1; SELECT 'DEDUPLICATE ON MergeTree'; DROP TABLE IF EXISTS tp; @@ -109,8 +109,8 @@ INSERT INTO tp SELECT number % 3, 2 FROM numbers(3); OPTIMIZE TABLE tp FINAL DEDUPLICATE BY type; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; +SETTINGS optimize_use_projections = 0, force_optimize_projection = 0; SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type -SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +SETTINGS optimize_use_projections = 1, force_optimize_projection = 1; From 00a27669df3f2153401d62b4d1681914bd0f440a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 20:22:14 +0200 Subject: [PATCH 17/41] Fix builds --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 69b0c095ad1..6263351897e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1258,6 +1258,7 @@ Chunk ArrowColumnToCHColumn::arrowColumnsToCHChunk(const NameToArrowColumn & nam .date_time_overflow_behavior = date_time_overflow_behavior, .allow_arrow_null_type = true, .skip_columns_with_unsupported_types = false + .allow_inferring_nullable_columns = true; }; Columns columns; From 902e7b6f29fe90007525f1f74adbf26e72b90bde Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 19 Aug 2024 23:58:48 +0000 Subject: [PATCH 18/41] ignore broken proj --- src/Storages/MergeTree/MergeTask.cpp | 11 ++++------- tests/integration/test_broken_projections/test.py | 2 +- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9ba09ea494a..8aac267d653 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -560,7 +560,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe for (const auto & part : global_ctx->future_part->parts) { auto it = part->getProjectionParts().find(projection.name); - if (it != part->getProjectionParts().end()) + if (it != part->getProjectionParts().end() && !it->second->is_broken) projection_parts.push_back(it->second); } if (projection_parts.size() == global_ctx->future_part->parts.size()) @@ -568,14 +568,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe global_ctx->projections_to_merge.push_back(&projection); global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); } - else if (projection_parts.empty()) - { - LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); - } else { - LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); - global_ctx->projections_to_rebuild.push_back(&projection); + chassert(projection_parts.size() < global_ctx->future_part->parts.size()); + LOG_DEBUG(ctx->log, "Projection {} is not merged because some parts don't have it", projection.name); + continue; } } diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 4ea018d3379..578ff42369c 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -191,7 +191,7 @@ def optimize(node, table, final, no_wait): if final: query += " FINAL" if no_wait: - query += " SETTINGS alter_sync=2" + query += " SETTINGS alter_sync=0" node.query(query) From 539d04c90f30efa0ef6435373ec8ffc4777aee78 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Aug 2024 20:00:23 +0000 Subject: [PATCH 19/41] Disable min_bytes_to_use_direct_io in some tests with Dynamic/JSON subcolumns because it's broken --- .../03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql | 2 ++ .../03036_dynamic_read_subcolumns_wide_merge_tree.sql | 2 ++ .../03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 | 2 ++ ...03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 | 2 ++ 4 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql index 61dc8fca01a..9e6e0652127 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql @@ -4,6 +4,8 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set allow_experimental_dynamic_type = 1; +set min_bytes_to_use_direct_io = 0; -- min_bytes_to_use_direct_io > 0 is broken + drop table if exists test; create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql index 5aac5f7b72f..44ceac1e5ad 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql @@ -4,6 +4,8 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set allow_experimental_dynamic_type = 1; +set min_bytes_to_use_direct_io = 0; -- min_bytes_to_use_direct_io > 0 is broken + drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 index ab4e0437c15..b31e57753c0 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -5,6 +5,8 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set session_timezone = 'UTC'; +set min_bytes_to_use_direct_io = 0; -- min_bytes_to_use_direct_io > 0 is broken + drop table if exists test; create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 index 3010fa0e2de..c0f34a8ea61 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -4,6 +4,8 @@ set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; +set min_bytes_to_use_direct_io = 0; -- min_bytes_to_use_direct_io > 0 is broken + create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; insert into test select number, '{}' from numbers(10000); From 1626589bb35eaca62804eabc5f64d5d55c998edf Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 00:48:23 +0300 Subject: [PATCH 20/41] feat: add ripeMD160 support --- src/Common/RipeMD160Hash.h | 186 +++++++++++++++++++++++++ src/Functions/FunctionsHashing.h | 16 +++ src/Functions/FunctionsHashingRipe.cpp | 21 +++ 3 files changed, 223 insertions(+) create mode 100644 src/Common/RipeMD160Hash.h create mode 100644 src/Functions/FunctionsHashingRipe.cpp diff --git a/src/Common/RipeMD160Hash.h b/src/Common/RipeMD160Hash.h new file mode 100644 index 00000000000..0ff97dc13b2 --- /dev/null +++ b/src/Common/RipeMD160Hash.h @@ -0,0 +1,186 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +class RIPEMD160 +{ +private: + UInt8 digest_bytes[20]; + + static constexpr UInt32 initial_digest[5] = {0x67452301UL, 0xefcdab89UL, 0x98badcfeUL, 0x10325476UL, 0xc3d2e1f0UL}; + + static constexpr UInt8 rho[16] = {0x7, 0x4, 0xd, 0x1, 0xa, 0x6, 0xf, 0x3, 0xc, 0x0, 0x9, 0x5, 0x2, 0xe, 0xb, 0x8}; + + static constexpr UInt8 shifts[80] + = {11, 14, 15, 12, 5, 8, 7, 9, 11, 13, 14, 15, 6, 7, 9, 8, 12, 13, 11, 15, 6, 9, 9, 7, 12, 15, 11, + 13, 7, 8, 7, 7, 13, 15, 14, 11, 7, 7, 6, 8, 13, 14, 13, 12, 5, 5, 6, 9, 14, 11, 12, 14, 8, 6, + 5, 5, 15, 12, 15, 14, 9, 9, 8, 6, 15, 12, 13, 13, 9, 5, 8, 6, 14, 11, 12, 11, 8, 6, 5, 5}; + + static constexpr UInt32 constants_left[5] = {0x00000000UL, 0x5a827999UL, 0x6ed9eba1UL, 0x8f1bbcdcUL, 0xa953fd4eUL}; + + + static ALWAYS_INLINE UInt32 make_le32(UInt32 x) noexcept + { + if constexpr (std::endian::native == std::endian::little) + { + return x; + } + else + { + return __builtin_bswap32(x); + } + } + + static constexpr UInt32 constants_right[5] = {0x50a28be6UL, 0x5c4dd124UL, 0x6d703ef3UL, 0x7a6d76e9UL, 0x00000000UL}; + + static constexpr UInt8 fns_left[5] = {1, 2, 3, 4, 5}; + static constexpr UInt8 fns_right[5] = {5, 4, 3, 2, 1}; + + static ALWAYS_INLINE UInt32 rol(UInt32 x, UInt32 n) noexcept { return (x << n) | (x >> (32 - n)); } + + static ALWAYS_INLINE UInt32 F_1(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ b ^ c); } + + static ALWAYS_INLINE UInt32 F_2(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a & b) | (~a & c)); } + + static ALWAYS_INLINE UInt32 F_3(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a | ~b) ^ c); } + + static ALWAYS_INLINE UInt32 F_4(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a & c) | (b & ~c)); } + + static ALWAYS_INLINE UInt32 F_5(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ (b | ~c)); } + + using FuncPtr = UInt32 (*)(UInt32, UInt32, UInt32); + + static constexpr FuncPtr funcs[5] = {F_1, F_2, F_3, F_4, F_5}; + + static ALWAYS_INLINE FuncPtr getFunction(UInt8 func) noexcept { return funcs[func - 1]; } + + void compute_line( + UInt32 * digest, + UInt32 * words, + const UInt32 * chunk, + UInt8 * index, + const UInt8 * sh, + const UInt32 * ks, + const UInt8 * fns) noexcept + { + std::memcpy(words, digest, 5 * sizeof(UInt32)); + for (UInt8 round = 0; round < 5; ++round) + { + UInt32 k = ks[round]; + UInt8 fn = fns[round]; + for (UInt8 j = 0; j < 16; ++j) + { + UInt32 tmp = getFunction(fn)(words[1], words[2], words[3]); + tmp += words[0] + le32toh(chunk[index[j]]) + k; + tmp = rol(tmp, sh[index[j]]) + words[4]; + words[0] = words[4]; + words[4] = words[3]; + words[3] = rol(words[2], 10); + words[2] = words[1]; + words[1] = tmp; + } + sh += 16; + UInt8 index_tmp[16]; + for (size_t i = 0; i < 16; ++i) + index_tmp[i] = rho[index[i]]; + std::memcpy(index, index_tmp, 16); + } + } + + /// Update the digest with the given chunk of data + void update(UInt32 * digest, const UInt32 * chunk) noexcept + { + UInt8 index[16]; + for (UInt8 i = 0; i < 16; ++i) + index[i] = i; + + UInt32 words_left[5]; + compute_line(digest, words_left, chunk, index, shifts, constants_left, fns_left); + + static constexpr UInt8 rho_index[16] = {5, 14, 7, 0, 9, 2, 11, 4, 13, 6, 15, 8, 1, 10, 3, 12}; + std::memcpy(index, rho_index, 16); + + UInt32 words_right[5]; + compute_line(digest, words_right, chunk, index, shifts, constants_right, fns_right); + + digest[0] += words_left[1] + words_right[2]; + digest[1] += words_left[2] + words_right[3]; + digest[2] += words_left[3] + words_right[4]; + digest[3] += words_left[4] + words_right[0]; + digest[4] += words_left[0] + words_right[1]; + + std::rotate(digest, digest + 1, digest + 5); + } + +public: + void hash(const UInt8 * data, size_t data_len) noexcept + { + UInt32 digest[5]; + for (size_t i = 0; i < 5; ++i) + digest[i] = make_le32(initial_digest[i]); + + const UInt8 * last_chunk_start = data + (data_len & (~0x3f)); + while (data < last_chunk_start) + { + update(digest, reinterpret_cast(data)); + data += 0x40; + } + + UInt8 last_chunk[0x40] = {}; + UInt8 leftover_size = data_len & 0x3f; + std::memcpy(last_chunk, data, leftover_size); + + last_chunk[leftover_size] = 0x80; + + if (leftover_size >= 0x38) + { + update(digest, reinterpret_cast(last_chunk)); + std::memset(last_chunk, 0, 0x38); + } + + UInt32 data_len_bits = static_cast(data_len << 3); + std::memcpy(&last_chunk[0x38], &data_len_bits, sizeof(data_len_bits)); + data_len_bits = static_cast(data_len >> 29); + std::memcpy(&last_chunk[0x3c], &data_len_bits, sizeof(data_len_bits)); + + update(digest, reinterpret_cast(last_chunk)); + + for (size_t i = 0; i < 5; ++i) + { + UInt32 digest_part = make_le32(digest[i]); + std::memcpy(digest_bytes + i * 4, &digest_part, 4); + } + } + + const UInt8 * get_digest_bytes() const noexcept { return digest_bytes; } +}; + + +inline UInt256 ripeMD160Hash(const char * data, const size_t size) noexcept +{ + RIPEMD160 ripe; + ripe.hash(reinterpret_cast(data), size); + + UInt8 digest[20]; + std::memcpy(digest, ripe.get_digest_bytes(), sizeof(digest)); + + std::reverse(digest, digest + sizeof(digest)); + + UInt256 res = 0; + std::memcpy(&res, digest, sizeof(digest)); + + return res; +} diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 95c54ac9528..8829e7c0479 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -190,6 +191,19 @@ T combineHashesFunc(T t1, T t2) return HashFunction::apply(reinterpret_cast(hashes), sizeof(hashes)); } +struct RipeMD160Impl +{ + static constexpr auto name = "ripeMD160"; + + using ReturnType = UInt256; + + static UInt256 apply(const char * begin, size_t size) { return ripeMD160Hash(begin, size); } + + static UInt256 combineHashes(UInt256 h1, UInt256 h2) { return combineHashesFunc(h1, h2); } + + static constexpr bool use_int_hash_for_pods = false; +}; + struct SipHash64Impl { @@ -1646,6 +1660,8 @@ using FunctionXxHash64 = FunctionAnyHash; using FunctionXXH3 = FunctionAnyHash; using FunctionWyHash64 = FunctionAnyHash; + +using FunctionRipeMD160Hash = FunctionAnyHash; } #pragma clang diagnostic pop diff --git a/src/Functions/FunctionsHashingRipe.cpp b/src/Functions/FunctionsHashingRipe.cpp new file mode 100644 index 00000000000..5b06b8ab924 --- /dev/null +++ b/src/Functions/FunctionsHashingRipe.cpp @@ -0,0 +1,21 @@ +#include "FunctionsHashing.h" + +#include + +/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. +namespace DB +{ +REGISTER_FUNCTION(HashingRipe) +{ + factory.registerFunction(FunctionDocumentation{ + .description = "RIPEMD-160 hash function, primarily used in Bitcoin address generation.", + .examples{{"", "SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog'));", R"( + ┌─hex(ripeMD160('The quick brown fox jumps over the lazy dog'))─┐ + │ 37F332F68DB77BD9D7EDD4969571AD671CF9DD3B │ + └───────────────────────────────────────────────────────────────┘ + )"}}, + .categories{"Hash"}}); +} +} From 5740df58b90f4b2f5532bd6b5ab5dc77a838a0a7 Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 01:17:40 +0300 Subject: [PATCH 21/41] feat: add test --- tests/queries/0_stateless/03222_ripeMD160.reference | 5 +++++ tests/queries/0_stateless/03222_ripeMD160.sql | 11 +++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03222_ripeMD160.reference create mode 100644 tests/queries/0_stateless/03222_ripeMD160.sql diff --git a/tests/queries/0_stateless/03222_ripeMD160.reference b/tests/queries/0_stateless/03222_ripeMD160.reference new file mode 100644 index 00000000000..f0db55dc8e1 --- /dev/null +++ b/tests/queries/0_stateless/03222_ripeMD160.reference @@ -0,0 +1,5 @@ +37F332F68DB77BD9D7EDD4969571AD671CF9DD3B +132072DF690933835EB8B6AD0B77E7B6F14ACAD7 +9C1185A5C5E9FC54612808977EE8F548B2258D31 +13920F39C93D503A0AC02EAB9AA8F672BC523ADA +3FEDF0C212CCFA54C0EBA676C8A8A2A10BC218BE diff --git a/tests/queries/0_stateless/03222_ripeMD160.sql b/tests/queries/0_stateless/03222_ripeMD160.sql new file mode 100644 index 00000000000..592f9f830dd --- /dev/null +++ b/tests/queries/0_stateless/03222_ripeMD160.sql @@ -0,0 +1,11 @@ +-- Ouput can be verified using: https://emn178.github.io/online-tools/ripemd-160/ + +SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog')); + +SELECT hex(ripeMD160('The quick brown fox jumps over the lazy cog')); + +SELECT hex(ripeMD160('')); + +SELECT hex(ripeMD160('CheREpaha1512')); + +SELECT hex(ripeMD160('A-very-long-string-that-should-be-hashed-using-ripeMD160')); From bb2b6600961e432d115c08964d65ade23740861a Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 02:11:08 +0300 Subject: [PATCH 22/41] fix: cosmetic --- src/Common/RipeMD160Hash.h | 151 +++++++++++++++++++------------------ 1 file changed, 78 insertions(+), 73 deletions(-) diff --git a/src/Common/RipeMD160Hash.h b/src/Common/RipeMD160Hash.h index 0ff97dc13b2..a759553278c 100644 --- a/src/Common/RipeMD160Hash.h +++ b/src/Common/RipeMD160Hash.h @@ -14,42 +14,33 @@ #include +/// https://homes.esat.kuleuven.be/~bosselae/ripemd160/pdf/AB-9601/AB-9601.pdf +/// https://en.wikipedia.org/wiki/RIPEMD -class RIPEMD160 +class RipeMD160 { private: + using FuncPtr = UInt32 (*)(UInt32, UInt32, UInt32); + + /// Stores the final 20-byte (160-bit) hash result UInt8 digest_bytes[20]; - static constexpr UInt32 initial_digest[5] = {0x67452301UL, 0xefcdab89UL, 0x98badcfeUL, 0x10325476UL, 0xc3d2e1f0UL}; + static constexpr UInt32 initial_hash_values[5] = {0x67452301UL, 0xEFCDAB89UL, 0x98BADCFEUL, 0x10325476UL, 0xC3D2E1F0UL}; - static constexpr UInt8 rho[16] = {0x7, 0x4, 0xd, 0x1, 0xa, 0x6, 0xf, 0x3, 0xc, 0x0, 0x9, 0x5, 0x2, 0xe, 0xb, 0x8}; + static constexpr UInt8 rho_order[16] = {0x7, 0x4, 0xD, 0x1, 0xA, 0x6, 0xF, 0x3, 0xC, 0x0, 0x9, 0x5, 0x2, 0xE, 0xB, 0x8}; - static constexpr UInt8 shifts[80] + static constexpr UInt8 shift_amounts[80] = {11, 14, 15, 12, 5, 8, 7, 9, 11, 13, 14, 15, 6, 7, 9, 8, 12, 13, 11, 15, 6, 9, 9, 7, 12, 15, 11, 13, 7, 8, 7, 7, 13, 15, 14, 11, 7, 7, 6, 8, 13, 14, 13, 12, 5, 5, 6, 9, 14, 11, 12, 14, 8, 6, 5, 5, 15, 12, 15, 14, 9, 9, 8, 6, 15, 12, 13, 13, 9, 5, 8, 6, 14, 11, 12, 11, 8, 6, 5, 5}; - static constexpr UInt32 constants_left[5] = {0x00000000UL, 0x5a827999UL, 0x6ed9eba1UL, 0x8f1bbcdcUL, 0xa953fd4eUL}; + static constexpr UInt32 left_round_constants[5] = {0x00000000UL, 0x5A827999UL, 0x6ED9EBA1UL, 0x8F1BBCDCUL, 0xA953FD4EUL}; - - static ALWAYS_INLINE UInt32 make_le32(UInt32 x) noexcept - { - if constexpr (std::endian::native == std::endian::little) - { - return x; - } - else - { - return __builtin_bswap32(x); - } - } + static constexpr UInt32 right_round_constants[5] = {0x50A28BE6UL, 0x5C4DD124UL, 0x6D703EF3UL, 0x7A6D76E9UL, 0x00000000UL}; - static constexpr UInt32 constants_right[5] = {0x50a28be6UL, 0x5c4dd124UL, 0x6d703ef3UL, 0x7a6d76e9UL, 0x00000000UL}; + static constexpr UInt8 left_function_order[5] = {1, 2, 3, 4, 5}; - static constexpr UInt8 fns_left[5] = {1, 2, 3, 4, 5}; - static constexpr UInt8 fns_right[5] = {5, 4, 3, 2, 1}; - - static ALWAYS_INLINE UInt32 rol(UInt32 x, UInt32 n) noexcept { return (x << n) | (x >> (32 - n)); } + static constexpr UInt8 right_function_order[5] = {5, 4, 3, 2, 1}; static ALWAYS_INLINE UInt32 F_1(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ b ^ c); } @@ -61,68 +52,82 @@ private: static ALWAYS_INLINE UInt32 F_5(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ (b | ~c)); } - using FuncPtr = UInt32 (*)(UInt32, UInt32, UInt32); + static constexpr FuncPtr hash_functions[5] = {F_1, F_2, F_3, F_4, F_5}; - static constexpr FuncPtr funcs[5] = {F_1, F_2, F_3, F_4, F_5}; + static ALWAYS_INLINE FuncPtr get_function(UInt8 function_id) noexcept { return hash_functions[function_id - 1]; } - static ALWAYS_INLINE FuncPtr getFunction(UInt8 func) noexcept { return funcs[func - 1]; } - - void compute_line( - UInt32 * digest, - UInt32 * words, - const UInt32 * chunk, - UInt8 * index, - const UInt8 * sh, - const UInt32 * ks, - const UInt8 * fns) noexcept + static ALWAYS_INLINE UInt32 convert_to_little_endian(UInt32 x) noexcept { - std::memcpy(words, digest, 5 * sizeof(UInt32)); + if constexpr (std::endian::native == std::endian::little) + { + return x; + } + else + { + return __builtin_bswap32(x); + } + } + + static ALWAYS_INLINE UInt32 rotate_left(UInt32 value, UInt32 shift) noexcept { return (value << shift) | (value >> (32 - shift)); } + + /// Performs one full pass (5 rounds) of RIPEMD-160 algorithm for one path (left or right) + void process_rounds( + UInt32 * current_digest, + UInt32 * temp_words, + const UInt32 * data_chunk, + UInt8 * index_order, + const UInt8 * shift_values, + const UInt32 * round_constants, + const UInt8 * function_order) noexcept + { + std::memcpy(temp_words, current_digest, 5 * sizeof(UInt32)); for (UInt8 round = 0; round < 5; ++round) { - UInt32 k = ks[round]; - UInt8 fn = fns[round]; + UInt32 k = round_constants[round]; + UInt8 fn = function_order[round]; for (UInt8 j = 0; j < 16; ++j) { - UInt32 tmp = getFunction(fn)(words[1], words[2], words[3]); - tmp += words[0] + le32toh(chunk[index[j]]) + k; - tmp = rol(tmp, sh[index[j]]) + words[4]; - words[0] = words[4]; - words[4] = words[3]; - words[3] = rol(words[2], 10); - words[2] = words[1]; - words[1] = tmp; + UInt32 temp_result = get_function(fn)(temp_words[1], temp_words[2], temp_words[3]); + temp_result += temp_words[0] + convert_to_little_endian(data_chunk[index_order[j]]) + k; + temp_result = rotate_left(temp_result, shift_values[index_order[j]]) + temp_words[4]; + temp_words[0] = temp_words[4]; + temp_words[4] = temp_words[3]; + temp_words[3] = rotate_left(temp_words[2], 10); + temp_words[2] = temp_words[1]; + temp_words[1] = temp_result; } - sh += 16; - UInt8 index_tmp[16]; + shift_values += 16; + UInt8 reordered_index[16]; for (size_t i = 0; i < 16; ++i) - index_tmp[i] = rho[index[i]]; - std::memcpy(index, index_tmp, 16); + reordered_index[i] = rho_order[index_order[i]]; + std::memcpy(index_order, reordered_index, 16); } } /// Update the digest with the given chunk of data - void update(UInt32 * digest, const UInt32 * chunk) noexcept + void update_digest(UInt32 * current_digest, const UInt32 * data_chunk) noexcept { - UInt8 index[16]; + UInt8 index_order[16]; for (UInt8 i = 0; i < 16; ++i) - index[i] = i; + index_order[i] = i; - UInt32 words_left[5]; - compute_line(digest, words_left, chunk, index, shifts, constants_left, fns_left); + UInt32 left_path_words[5]; + process_rounds(current_digest, left_path_words, data_chunk, index_order, shift_amounts, left_round_constants, left_function_order); - static constexpr UInt8 rho_index[16] = {5, 14, 7, 0, 9, 2, 11, 4, 13, 6, 15, 8, 1, 10, 3, 12}; - std::memcpy(index, rho_index, 16); + static constexpr UInt8 rho_reordered_index[16] = {5, 14, 7, 0, 9, 2, 11, 4, 13, 6, 15, 8, 1, 10, 3, 12}; + std::memcpy(index_order, rho_reordered_index, 16); - UInt32 words_right[5]; - compute_line(digest, words_right, chunk, index, shifts, constants_right, fns_right); + UInt32 right_path_words[5]; + process_rounds( + current_digest, right_path_words, data_chunk, index_order, shift_amounts, right_round_constants, right_function_order); - digest[0] += words_left[1] + words_right[2]; - digest[1] += words_left[2] + words_right[3]; - digest[2] += words_left[3] + words_right[4]; - digest[3] += words_left[4] + words_right[0]; - digest[4] += words_left[0] + words_right[1]; + current_digest[0] += left_path_words[1] + right_path_words[2]; + current_digest[1] += left_path_words[2] + right_path_words[3]; + current_digest[2] += left_path_words[3] + right_path_words[4]; + current_digest[3] += left_path_words[4] + right_path_words[0]; + current_digest[4] += left_path_words[0] + right_path_words[1]; - std::rotate(digest, digest + 1, digest + 5); + std::rotate(current_digest, current_digest + 1, current_digest + 5); } public: @@ -130,37 +135,37 @@ public: { UInt32 digest[5]; for (size_t i = 0; i < 5; ++i) - digest[i] = make_le32(initial_digest[i]); + digest[i] = convert_to_little_endian(initial_hash_values[i]); - const UInt8 * last_chunk_start = data + (data_len & (~0x3f)); + const UInt8 * last_chunk_start = data + (data_len & (~0x3F)); while (data < last_chunk_start) { - update(digest, reinterpret_cast(data)); + update_digest(digest, reinterpret_cast(data)); data += 0x40; } UInt8 last_chunk[0x40] = {}; - UInt8 leftover_size = data_len & 0x3f; + UInt8 leftover_size = data_len & 0x3F; std::memcpy(last_chunk, data, leftover_size); last_chunk[leftover_size] = 0x80; if (leftover_size >= 0x38) { - update(digest, reinterpret_cast(last_chunk)); + update_digest(digest, reinterpret_cast(last_chunk)); std::memset(last_chunk, 0, 0x38); } UInt32 data_len_bits = static_cast(data_len << 3); std::memcpy(&last_chunk[0x38], &data_len_bits, sizeof(data_len_bits)); data_len_bits = static_cast(data_len >> 29); - std::memcpy(&last_chunk[0x3c], &data_len_bits, sizeof(data_len_bits)); + std::memcpy(&last_chunk[0x3C], &data_len_bits, sizeof(data_len_bits)); - update(digest, reinterpret_cast(last_chunk)); + update_digest(digest, reinterpret_cast(last_chunk)); for (size_t i = 0; i < 5; ++i) { - UInt32 digest_part = make_le32(digest[i]); + UInt32 digest_part = convert_to_little_endian(digest[i]); std::memcpy(digest_bytes + i * 4, &digest_part, 4); } } @@ -171,7 +176,7 @@ public: inline UInt256 ripeMD160Hash(const char * data, const size_t size) noexcept { - RIPEMD160 ripe; + RipeMD160 ripe; ripe.hash(reinterpret_cast(data), size); UInt8 digest[20]; From a0d29c812c4da7c7e8d15e798e141e7e6be910b9 Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 02:47:23 +0300 Subject: [PATCH 23/41] fix: cosmetic --- src/Common/RipeMD160Hash.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/RipeMD160Hash.h b/src/Common/RipeMD160Hash.h index a759553278c..b6488225974 100644 --- a/src/Common/RipeMD160Hash.h +++ b/src/Common/RipeMD160Hash.h @@ -34,7 +34,7 @@ private: 13, 7, 8, 7, 7, 13, 15, 14, 11, 7, 7, 6, 8, 13, 14, 13, 12, 5, 5, 6, 9, 14, 11, 12, 14, 8, 6, 5, 5, 15, 12, 15, 14, 9, 9, 8, 6, 15, 12, 13, 13, 9, 5, 8, 6, 14, 11, 12, 11, 8, 6, 5, 5}; - static constexpr UInt32 left_round_constants[5] = {0x00000000UL, 0x5A827999UL, 0x6ED9EBA1UL, 0x8F1BBCDCUL, 0xA953FD4EUL}; + static constexpr UInt32 left_round_constants[5] = {0x00000000UL, 0x5A827999UL, 0x6ED9EBA1UL, 0x8F1BBCDCUL, 0xA953FD4EUL}; static constexpr UInt32 right_round_constants[5] = {0x50A28BE6UL, 0x5C4DD124UL, 0x6D703EF3UL, 0x7A6D76E9UL, 0x00000000UL}; From a387807c8429af1c9ed471b80ad317c9171b40a4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Aug 2024 15:14:51 +0200 Subject: [PATCH 24/41] Fix build --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 6263351897e..c9ca9efabee 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1257,8 +1257,8 @@ Chunk ArrowColumnToCHColumn::arrowColumnsToCHChunk(const NameToArrowColumn & nam .format_name = format_name, .date_time_overflow_behavior = date_time_overflow_behavior, .allow_arrow_null_type = true, - .skip_columns_with_unsupported_types = false - .allow_inferring_nullable_columns = true; + .skip_columns_with_unsupported_types = false, + .allow_inferring_nullable_columns = true }; Columns columns; From 6db7b995439f873f9cd33d07019ea939ec51a3b7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Aug 2024 15:42:46 +0200 Subject: [PATCH 25/41] Increase connectTimeoutMs IMDS connection timeout to 50ms to avoid failures in CI --- src/IO/S3/Credentials.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index d6f7542da6b..fab3a0111eb 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -787,7 +787,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. /// But the connection timeout should be small because there is the case when there is no IMDS at all, /// like outside of the cloud, on your own machines. - aws_client_configuration.connectTimeoutMs = 10; + aws_client_configuration.connectTimeoutMs = 50; aws_client_configuration.requestTimeoutMs = 1000; aws_client_configuration.retryStrategy = std::make_shared(1, 1000); From 3fd50ed856a9767094f19f08c93401fc4a5a80eb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Aug 2024 16:23:37 +0200 Subject: [PATCH 26/41] Fix flaky test test_distributed_replica_max_ignored_errors --- tests/integration/test_distributed_load_balancing/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index d61cca6ce12..a913c2ebb49 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -200,7 +200,6 @@ def test_distributed_replica_max_ignored_errors(): "connect_timeout": 2, "receive_timeout": 2, "send_timeout": 2, - "idle_connection_timeout": 2, "tcp_keep_alive_timeout": 2, "distributed_replica_max_ignored_errors": 0, "distributed_replica_error_half_life": 60, From 74d8971432158a312777dcfba229513bfd876acc Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 22:06:52 +0300 Subject: [PATCH 27/41] fix: use OpenSSL RIPEMD160 impl --- src/Common/RipeMD160Hash.h | 191 ------------------------------- src/Functions/FunctionsHashing.h | 23 +++- 2 files changed, 19 insertions(+), 195 deletions(-) delete mode 100644 src/Common/RipeMD160Hash.h diff --git a/src/Common/RipeMD160Hash.h b/src/Common/RipeMD160Hash.h deleted file mode 100644 index b6488225974..00000000000 --- a/src/Common/RipeMD160Hash.h +++ /dev/null @@ -1,191 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -/// https://homes.esat.kuleuven.be/~bosselae/ripemd160/pdf/AB-9601/AB-9601.pdf -/// https://en.wikipedia.org/wiki/RIPEMD - -class RipeMD160 -{ -private: - using FuncPtr = UInt32 (*)(UInt32, UInt32, UInt32); - - /// Stores the final 20-byte (160-bit) hash result - UInt8 digest_bytes[20]; - - static constexpr UInt32 initial_hash_values[5] = {0x67452301UL, 0xEFCDAB89UL, 0x98BADCFEUL, 0x10325476UL, 0xC3D2E1F0UL}; - - static constexpr UInt8 rho_order[16] = {0x7, 0x4, 0xD, 0x1, 0xA, 0x6, 0xF, 0x3, 0xC, 0x0, 0x9, 0x5, 0x2, 0xE, 0xB, 0x8}; - - static constexpr UInt8 shift_amounts[80] - = {11, 14, 15, 12, 5, 8, 7, 9, 11, 13, 14, 15, 6, 7, 9, 8, 12, 13, 11, 15, 6, 9, 9, 7, 12, 15, 11, - 13, 7, 8, 7, 7, 13, 15, 14, 11, 7, 7, 6, 8, 13, 14, 13, 12, 5, 5, 6, 9, 14, 11, 12, 14, 8, 6, - 5, 5, 15, 12, 15, 14, 9, 9, 8, 6, 15, 12, 13, 13, 9, 5, 8, 6, 14, 11, 12, 11, 8, 6, 5, 5}; - - static constexpr UInt32 left_round_constants[5] = {0x00000000UL, 0x5A827999UL, 0x6ED9EBA1UL, 0x8F1BBCDCUL, 0xA953FD4EUL}; - - static constexpr UInt32 right_round_constants[5] = {0x50A28BE6UL, 0x5C4DD124UL, 0x6D703EF3UL, 0x7A6D76E9UL, 0x00000000UL}; - - static constexpr UInt8 left_function_order[5] = {1, 2, 3, 4, 5}; - - static constexpr UInt8 right_function_order[5] = {5, 4, 3, 2, 1}; - - static ALWAYS_INLINE UInt32 F_1(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ b ^ c); } - - static ALWAYS_INLINE UInt32 F_2(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a & b) | (~a & c)); } - - static ALWAYS_INLINE UInt32 F_3(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a | ~b) ^ c); } - - static ALWAYS_INLINE UInt32 F_4(UInt32 a, UInt32 b, UInt32 c) noexcept { return ((a & c) | (b & ~c)); } - - static ALWAYS_INLINE UInt32 F_5(UInt32 a, UInt32 b, UInt32 c) noexcept { return (a ^ (b | ~c)); } - - static constexpr FuncPtr hash_functions[5] = {F_1, F_2, F_3, F_4, F_5}; - - static ALWAYS_INLINE FuncPtr get_function(UInt8 function_id) noexcept { return hash_functions[function_id - 1]; } - - static ALWAYS_INLINE UInt32 convert_to_little_endian(UInt32 x) noexcept - { - if constexpr (std::endian::native == std::endian::little) - { - return x; - } - else - { - return __builtin_bswap32(x); - } - } - - static ALWAYS_INLINE UInt32 rotate_left(UInt32 value, UInt32 shift) noexcept { return (value << shift) | (value >> (32 - shift)); } - - /// Performs one full pass (5 rounds) of RIPEMD-160 algorithm for one path (left or right) - void process_rounds( - UInt32 * current_digest, - UInt32 * temp_words, - const UInt32 * data_chunk, - UInt8 * index_order, - const UInt8 * shift_values, - const UInt32 * round_constants, - const UInt8 * function_order) noexcept - { - std::memcpy(temp_words, current_digest, 5 * sizeof(UInt32)); - for (UInt8 round = 0; round < 5; ++round) - { - UInt32 k = round_constants[round]; - UInt8 fn = function_order[round]; - for (UInt8 j = 0; j < 16; ++j) - { - UInt32 temp_result = get_function(fn)(temp_words[1], temp_words[2], temp_words[3]); - temp_result += temp_words[0] + convert_to_little_endian(data_chunk[index_order[j]]) + k; - temp_result = rotate_left(temp_result, shift_values[index_order[j]]) + temp_words[4]; - temp_words[0] = temp_words[4]; - temp_words[4] = temp_words[3]; - temp_words[3] = rotate_left(temp_words[2], 10); - temp_words[2] = temp_words[1]; - temp_words[1] = temp_result; - } - shift_values += 16; - UInt8 reordered_index[16]; - for (size_t i = 0; i < 16; ++i) - reordered_index[i] = rho_order[index_order[i]]; - std::memcpy(index_order, reordered_index, 16); - } - } - - /// Update the digest with the given chunk of data - void update_digest(UInt32 * current_digest, const UInt32 * data_chunk) noexcept - { - UInt8 index_order[16]; - for (UInt8 i = 0; i < 16; ++i) - index_order[i] = i; - - UInt32 left_path_words[5]; - process_rounds(current_digest, left_path_words, data_chunk, index_order, shift_amounts, left_round_constants, left_function_order); - - static constexpr UInt8 rho_reordered_index[16] = {5, 14, 7, 0, 9, 2, 11, 4, 13, 6, 15, 8, 1, 10, 3, 12}; - std::memcpy(index_order, rho_reordered_index, 16); - - UInt32 right_path_words[5]; - process_rounds( - current_digest, right_path_words, data_chunk, index_order, shift_amounts, right_round_constants, right_function_order); - - current_digest[0] += left_path_words[1] + right_path_words[2]; - current_digest[1] += left_path_words[2] + right_path_words[3]; - current_digest[2] += left_path_words[3] + right_path_words[4]; - current_digest[3] += left_path_words[4] + right_path_words[0]; - current_digest[4] += left_path_words[0] + right_path_words[1]; - - std::rotate(current_digest, current_digest + 1, current_digest + 5); - } - -public: - void hash(const UInt8 * data, size_t data_len) noexcept - { - UInt32 digest[5]; - for (size_t i = 0; i < 5; ++i) - digest[i] = convert_to_little_endian(initial_hash_values[i]); - - const UInt8 * last_chunk_start = data + (data_len & (~0x3F)); - while (data < last_chunk_start) - { - update_digest(digest, reinterpret_cast(data)); - data += 0x40; - } - - UInt8 last_chunk[0x40] = {}; - UInt8 leftover_size = data_len & 0x3F; - std::memcpy(last_chunk, data, leftover_size); - - last_chunk[leftover_size] = 0x80; - - if (leftover_size >= 0x38) - { - update_digest(digest, reinterpret_cast(last_chunk)); - std::memset(last_chunk, 0, 0x38); - } - - UInt32 data_len_bits = static_cast(data_len << 3); - std::memcpy(&last_chunk[0x38], &data_len_bits, sizeof(data_len_bits)); - data_len_bits = static_cast(data_len >> 29); - std::memcpy(&last_chunk[0x3C], &data_len_bits, sizeof(data_len_bits)); - - update_digest(digest, reinterpret_cast(last_chunk)); - - for (size_t i = 0; i < 5; ++i) - { - UInt32 digest_part = convert_to_little_endian(digest[i]); - std::memcpy(digest_bytes + i * 4, &digest_part, 4); - } - } - - const UInt8 * get_digest_bytes() const noexcept { return digest_bytes; } -}; - - -inline UInt256 ripeMD160Hash(const char * data, const size_t size) noexcept -{ - RipeMD160 ripe; - ripe.hash(reinterpret_cast(data), size); - - UInt8 digest[20]; - std::memcpy(digest, ripe.get_digest_bytes(), sizeof(digest)); - - std::reverse(digest, digest + sizeof(digest)); - - UInt256 res = 0; - std::memcpy(&res, digest, sizeof(digest)); - - return res; -} diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 8829e7c0479..5111ee2bd90 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -14,13 +14,14 @@ #include #include -#include #include #include #include #if USE_SSL +# include # include +# include #endif #include @@ -194,12 +195,26 @@ T combineHashesFunc(T t1, T t2) struct RipeMD160Impl { static constexpr auto name = "ripeMD160"; - using ReturnType = UInt256; - static UInt256 apply(const char * begin, size_t size) { return ripeMD160Hash(begin, size); } + static UInt256 apply(const char * begin, size_t size) + { + UInt8 digest[RIPEMD160_DIGEST_LENGTH]; - static UInt256 combineHashes(UInt256 h1, UInt256 h2) { return combineHashesFunc(h1, h2); } + RIPEMD160(reinterpret_cast(begin), size, reinterpret_cast(digest)); + + std::reverse(digest, digest + RIPEMD160_DIGEST_LENGTH); + + UInt256 res = 0; + std::memcpy(&res, digest, RIPEMD160_DIGEST_LENGTH); + + return res; + } + + static UInt256 combineHashes(UInt256 h1, UInt256 h2) + { + return combineHashesFunc(h1, h2); + } static constexpr bool use_int_hash_for_pods = false; }; From dfe0beb53b4f0d1da50bf04e9c9e3e06f8b29ad2 Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 22:46:29 +0300 Subject: [PATCH 28/41] feat: add docs --- .../sql-reference/functions/hash-functions.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 7c977e7d6dc..d610e23fdda 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -688,6 +688,36 @@ SELECT kostikConsistentHash(16045690984833335023, 2); └───────────────────────────────────────────────┘ ``` +## ripeMD160 + +**Syntax** + +```sql +ripeMD160('input') +``` + +**Parameters** + +- `input`: Input string. [String](../data-types/string.md) + +**Returned value** + +- A [UInt256](../data-types/int-uint.md) hash value of type [FixedString(20)](../data-types/fixedstring.md). + +**Example** + +Query: + +```sql +SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog')); +``` + +```response +┌─hex(ripeMD160('The quick brown fox jumps over the lazy dog'))─┐ +│ 37F332F68DB77BD9D7EDD4969571AD671CF9DD3B │ +└───────────────────────────────────────────────────────────────┘ +``` + ## murmurHash2_32, murmurHash2_64 Produces a [MurmurHash2](https://github.com/aappleby/smhasher) hash value. From 7f15f61426d07561ad0e24d946ac126961038a0c Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 22:46:55 +0300 Subject: [PATCH 29/41] feat: add docs --- .../sql-reference/functions/hash-functions.md | 6 +++- .../sql-reference/functions/hash-functions.md | 32 +++++++++++++++++++ 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index d610e23fdda..9b7ac8af0e3 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -689,6 +689,9 @@ SELECT kostikConsistentHash(16045690984833335023, 2); ``` ## ripeMD160 +Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash of a string and returns the resulting set of bytes as [FixedString](../data-types/fixedstring.md). + + **Syntax** @@ -702,9 +705,10 @@ ripeMD160('input') **Returned value** -- A [UInt256](../data-types/int-uint.md) hash value of type [FixedString(20)](../data-types/fixedstring.md). +- A [UInt256](../data-types/int-uint.md) hash value **Example** +Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. Query: diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 98b6d8d4b17..66d77e66972 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -124,6 +124,38 @@ SELECT hex(sipHash128('foo', '\x01', 3)); └──────────────────────────────────┘ ``` +## ripeMD160 +Генерирует [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) хеш строки и возвращает полученный набор байт в виде [FixedString](../data-types/fixedstring.md). + +**Синтаксис** + +```sql +ripeMD160('input') +``` + +**Аргументы** + +- `input`: Строка [String](../data-types/string.md) + +**Возвращаемое значение** + +- [UInt256](../data-types/int-uint.md) хеш-значение + +**Пример** +Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой + +Запрос: + +```sql +SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog')); +``` +Результат: +```response +┌─hex(ripeMD160('The quick brown fox jumps over the lazy dog'))─┐ +│ 37F332F68DB77BD9D7EDD4969571AD671CF9DD3B │ +└───────────────────────────────────────────────────────────────┘ +``` + ## cityHash64 {#cityhash64} Генерирует 64-х битное значение [CityHash](https://github.com/google/cityhash). From 0f3c7ae8c202f475fe55f33f45e9bca92155d52c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:15:16 +0200 Subject: [PATCH 30/41] feat: add docs --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ffd9fae7f45..308e285c4bd 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2942 +personal_ws-1.1 en 2983 AArch ACLs ALTERs @@ -957,6 +957,7 @@ ThreadPoolRemoteFSReaderThreads ThreadPoolRemoteFSReaderThreadsActive ThreadsActive ThreadsInOvercommitTracker +TimeSeries Timeunit TinyLog Tkachenko @@ -1098,12 +1099,12 @@ addressToLineWithInlines addressToSymbol adviced agg +aggThrow aggregatefunction aggregatingmergetree aggregatio aggretate aggthrow -aggThrow aiochclient allocator alphaTokens @@ -1875,8 +1876,8 @@ joinGet joinGetOrNull json jsonMergePatch -jsonasstring jsonasobject +jsonasstring jsoncolumns jsoncolumnsmonoblock jsoncompact @@ -1917,8 +1918,8 @@ kurtSamp kurtosis kurtpop kurtsamp -laion lagInFrame +laion lang laravel largestTriangleThreeBuckets @@ -2020,7 +2021,6 @@ maxMap maxintersections maxintersectionsposition maxmap -minMappedArrays maxmind mdadm meanZTest @@ -2213,8 +2213,8 @@ parseReadableSizeOrZero parseTimeDelta parseable parsers -partitionId partitionID +partitionId pathFull pclmulqdq pcre @@ -2443,6 +2443,7 @@ rewritable rightPad rightPadUTF rightUTF +ripeMD risc riscv ro @@ -2694,7 +2695,6 @@ themself threadpool throwIf timeDiff -TimeSeries timeSeriesData timeSeriesMetrics timeSeriesTags From 54caf1f84e3c3b5076adf29b49f4ee548f243091 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 31/41] fix: wrap in conditional preprocessor directives --- src/Functions/FunctionsHashing.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 5111ee2bd90..ec39cf1e2cf 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -192,6 +192,7 @@ T combineHashesFunc(T t1, T t2) return HashFunction::apply(reinterpret_cast(hashes), sizeof(hashes)); } +#if USE_SSL struct RipeMD160Impl { static constexpr auto name = "ripeMD160"; @@ -218,7 +219,7 @@ struct RipeMD160Impl static constexpr bool use_int_hash_for_pods = false; }; - +#endif struct SipHash64Impl { @@ -1647,6 +1648,7 @@ using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL using FunctionHalfMD5 = FunctionAnyHash; +using FunctionRipeMD160Hash = FunctionAnyHash; #endif using FunctionSipHash128 = FunctionAnyHash; using FunctionSipHash128Keyed = FunctionAnyHash; @@ -1676,7 +1678,6 @@ using FunctionXXH3 = FunctionAnyHash; using FunctionWyHash64 = FunctionAnyHash; -using FunctionRipeMD160Hash = FunctionAnyHash; } #pragma clang diagnostic pop From 837f2bba8a136170b6aa8800b6b30849a9310e5f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:23:45 +0200 Subject: [PATCH 32/41] init --- .../0_stateless/00080_show_tables_and_system_tables.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index a58f9ddb0ac..02e3645ece0 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -6,8 +6,8 @@ CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = TinyLog; CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.B (A UInt8) ENGINE = TinyLog; -SHOW TABLES from {CLICKHOUSE_DATABASE:Identifier}; -SHOW TABLES in system where engine like '%System%' and name in ('numbers', 'one'); +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; +SHOW TABLES IN system WHERE engine LIKE '%System%' AND name IN ('numbers', 'one') AND database = 'system'; SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = currentDatabase() ORDER BY name FORMAT TSVRaw; @@ -16,7 +16,7 @@ SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; -SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' and database=currentDatabase(); +SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' AND database=currentDatabase(); DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; From 2a32207e9ee44d52d6fbca7313d847b4eef1c4fb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 33/41] fix: wrap in conditional preprocessor directives --- src/Functions/FunctionsHashingRipe.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionsHashingRipe.cpp b/src/Functions/FunctionsHashingRipe.cpp index 5b06b8ab924..315296b7690 100644 --- a/src/Functions/FunctionsHashingRipe.cpp +++ b/src/Functions/FunctionsHashingRipe.cpp @@ -7,6 +7,7 @@ /// due to excessive resource consumption. namespace DB { +#if USE_SSL REGISTER_FUNCTION(HashingRipe) { factory.registerFunction(FunctionDocumentation{ @@ -18,4 +19,5 @@ REGISTER_FUNCTION(HashingRipe) )"}}, .categories{"Hash"}}); } +#endif } From ef9fbe3006b3023bf47e3a0109490d166071c2aa Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 34/41] fix: disable running test in fasttest due to missing OpenSSL --- tests/queries/0_stateless/03222_ripeMD160.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03222_ripeMD160.sql b/tests/queries/0_stateless/03222_ripeMD160.sql index 592f9f830dd..9d418376a20 100644 --- a/tests/queries/0_stateless/03222_ripeMD160.sql +++ b/tests/queries/0_stateless/03222_ripeMD160.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest -- Ouput can be verified using: https://emn178.github.io/online-tools/ripemd-160/ SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog')); From a2ff8e4384f04b2e57d3de93a1ba63971f94794e Mon Sep 17 00:00:00 2001 From: Dergousov Date: Thu, 22 Aug 2024 20:44:52 +0300 Subject: [PATCH 35/41] fix: correct return type inconsistencies in docs --- docs/en/sql-reference/functions/hash-functions.md | 5 ++--- docs/ru/sql-reference/functions/hash-functions.md | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 9b7ac8af0e3..cd1c85b5f4c 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -689,9 +689,8 @@ SELECT kostikConsistentHash(16045690984833335023, 2); ``` ## ripeMD160 -Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash of a string and returns the resulting set of bytes as [FixedString](../data-types/fixedstring.md). - +Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash value. **Syntax** @@ -705,7 +704,7 @@ ripeMD160('input') **Returned value** -- A [UInt256](../data-types/int-uint.md) hash value +- A [UInt256]((../data-types/int-uint.md)) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 66d77e66972..b7adcfc1829 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -125,7 +125,8 @@ SELECT hex(sipHash128('foo', '\x01', 3)); ``` ## ripeMD160 -Генерирует [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) хеш строки и возвращает полученный набор байт в виде [FixedString](../data-types/fixedstring.md). + +Генерирует [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) хеш строки. **Синтаксис** @@ -139,7 +140,7 @@ ripeMD160('input') **Возвращаемое значение** -- [UInt256](../data-types/int-uint.md) хеш-значение +- [UInt256](../data-types/int-uint.md), где 160-битный хеш RIPEMD-160 хранится в первых 20 байтах. Оставшиеся 12 байт заполняются нулями. **Пример** Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой From 69f6ea5083f1686becce4ca9fcf47d1404f2d3ed Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 22:07:02 +0200 Subject: [PATCH 36/41] Update docs/en/sql-reference/functions/hash-functions.md --- docs/en/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index cd1c85b5f4c..55126640e34 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -704,7 +704,7 @@ ripeMD160('input') **Returned value** -- A [UInt256]((../data-types/int-uint.md)) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. +- A [UInt256](../data-types/int-uint.md) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. From 8c4329964f597b1eb8139990a41360243f9337f9 Mon Sep 17 00:00:00 2001 From: Maxim Dergousov Date: Fri, 23 Aug 2024 12:50:18 +0300 Subject: [PATCH 37/41] small cosmetic changes in docs --- docs/en/sql-reference/functions/hash-functions.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 55126640e34..908e288cf59 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -695,7 +695,7 @@ Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash value. **Syntax** ```sql -ripeMD160('input') +ripeMD160(input) ``` **Parameters** @@ -707,6 +707,7 @@ ripeMD160('input') - A [UInt256](../data-types/int-uint.md) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** + Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. Query: From 8cc5d766b5d70f22646e6dc2832f806736c76311 Mon Sep 17 00:00:00 2001 From: Maxim Dergousov Date: Fri, 23 Aug 2024 12:52:55 +0300 Subject: [PATCH 38/41] small cosmetic changes in docs --- docs/ru/sql-reference/functions/hash-functions.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index b7adcfc1829..d7b90b09122 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -131,7 +131,7 @@ SELECT hex(sipHash128('foo', '\x01', 3)); **Синтаксис** ```sql -ripeMD160('input') +ripeMD160(input) ``` **Аргументы** @@ -143,6 +143,7 @@ ripeMD160('input') - [UInt256](../data-types/int-uint.md), где 160-битный хеш RIPEMD-160 хранится в первых 20 байтах. Оставшиеся 12 байт заполняются нулями. **Пример** + Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой Запрос: From 327828777999447756823fe1e6c99cb867896154 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 15:26:45 +0000 Subject: [PATCH 39/41] fix --- .../Optimizations/optimizeReadInOrder.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 5df7d7b4e82..9dc4992cddd 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -255,7 +255,7 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi /// Add more functions to fixed columns. /// Functions result is fixed if all arguments are fixed or constants. -void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) +void enrichFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { struct Frame { @@ -300,20 +300,20 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { if (frame.node->function_base->isDeterministicInScopeOfQuery()) { - //std::cerr << "*** enreachFixedColumns check " << frame.node->result_name << std::endl; + //std::cerr << "*** enrichFixedColumns check " << frame.node->result_name << std::endl; bool all_args_fixed_or_const = true; for (const auto * child : frame.node->children) { if (!child->column && !fixed_columns.contains(child)) { - //std::cerr << "*** enreachFixedColumns fail " << child->result_name << ' ' << static_cast(child) << std::endl; + //std::cerr << "*** enrichFixedColumns fail " << child->result_name << ' ' << static_cast(child) << std::endl; all_args_fixed_or_const = false; } } if (all_args_fixed_or_const) { - //std::cerr << "*** enreachFixedColumns add " << frame.node->result_name << ' ' << static_cast(frame.node) << std::endl; + //std::cerr << "*** enrichFixedColumns add " << frame.node->result_name << ' ' << static_cast(frame.node) << std::endl; fixed_columns.insert(frame.node); } } @@ -357,7 +357,7 @@ InputOrderInfoPtr buildInputOrderInfo( } } - enreachFixedColumns(sorting_key_dag, fixed_key_columns); + enrichFixedColumns(sorting_key_dag, fixed_key_columns); } /// This is a result direction we will read from MergeTree @@ -530,7 +530,7 @@ AggregationInputOrder buildInputOrderInfo( } } - enreachFixedColumns(sorting_key_dag, fixed_key_columns); + enrichFixedColumns(sorting_key_dag, fixed_key_columns); for (const auto * output : dag->getOutputs()) { @@ -804,7 +804,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n buildSortingDAG(node, dag, fixed_columns, limit); if (dag && !fixed_columns.empty()) - enreachFixedColumns(*dag, fixed_columns); + enrichFixedColumns(*dag, fixed_columns); if (auto * reading = typeid_cast(reading_node->step.get())) { @@ -858,7 +858,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl buildSortingDAG(node, dag, fixed_columns, limit); if (dag && !fixed_columns.empty()) - enreachFixedColumns(*dag, fixed_columns); + enrichFixedColumns(*dag, fixed_columns); if (auto * reading = typeid_cast(reading_node->step.get())) { From eb94847ed9c5d5834c05560fb5e60fc7ab1224ff Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 23 Aug 2024 18:14:02 +0200 Subject: [PATCH 40/41] Fix --- .../test/integration/runner/requirements.txt | 2 + .../DataLakes/DeltaLakeMetadata.cpp | 13 +-- tests/integration/test_storage_delta/test.py | 95 +++++++++++++++++++ 3 files changed, 104 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 428986b5562..2c9df73ccca 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -112,3 +112,5 @@ wadllib==1.3.6 websocket-client==0.59.0 wheel==0.37.1 zipp==1.0.0 +deltalake==0.16.0 + diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 7aadba18817..9b972711cb1 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -425,8 +425,9 @@ struct DeltaLakeMetadataImpl { auto field = fields->getObject(static_cast(i)); element_names.push_back(field->getValue("name")); - auto required = field->getValue("required"); - element_types.push_back(getFieldType(field, "type", required)); + + auto is_nullable = field->getValue("nullable"); + element_types.push_back(getFieldType(field, "type", is_nullable)); } return std::make_shared(element_types, element_names); @@ -434,16 +435,16 @@ struct DeltaLakeMetadataImpl if (type_name == "array") { - bool is_nullable = type->getValue("containsNull"); - auto element_type = getFieldType(type, "elementType", is_nullable); + bool element_nullable = type->getValue("containsNull"); + auto element_type = getFieldType(type, "elementType", element_nullable); return std::make_shared(element_type); } if (type_name == "map") { - bool is_nullable = type->getValue("containsNull"); auto key_type = getFieldType(type, "keyType", /* is_nullable */false); - auto value_type = getFieldType(type, "valueType", is_nullable); + bool value_nullable = type->getValue("valueContainsNull"); + auto value_type = getFieldType(type, "valueType", value_nullable); return std::make_shared(key_type, value_type); } diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 75a4b6cc221..d000875d8d1 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -29,6 +29,9 @@ from datetime import datetime from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject +import pyarrow as pa +import pyarrow.parquet as pq +from deltalake.writer import write_deltalake from helpers.s3_tools import ( prepare_s3_bucket, @@ -728,3 +731,95 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) + + +def test_complex_types(started_cluster): + node = started_cluster.instances["node1"] + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + + schema = pa.schema( + [ + ("id", pa.int32()), + ("name", pa.string()), + ( + "address", + pa.struct( + [ + ("street", pa.string()), + ("city", pa.string()), + ("state", pa.string()), + ] + ), + ), + ("interests", pa.list_(pa.string())), + ( + "metadata", + pa.map_( + pa.string(), pa.string() + ), # Map with string keys and string values + ), + ] + ) + + # Create sample data + data = [ + pa.array([1, 2, 3], type=pa.int32()), + pa.array(["John Doe", "Jane Smith", "Jake Johnson"], type=pa.string()), + pa.array( + [ + {"street": "123 Elm St", "city": "Springfield", "state": "IL"}, + {"street": "456 Maple St", "city": "Shelbyville", "state": "IL"}, + {"street": "789 Oak St", "city": "Ogdenville", "state": "IL"}, + ], + type=schema.field("address").type, + ), + pa.array( + [ + pa.array(["dancing", "coding", "hiking"]), + pa.array(["dancing", "coding", "hiking"]), + pa.array(["dancing", "coding", "hiking"]), + ], + type=schema.field("interests").type, + ), + pa.array( + [ + {"key1": "value1", "key2": "value2"}, + {"key1": "value3", "key2": "value4"}, + {"key1": "value5", "key2": "value6"}, + ], + type=schema.field("metadata").type, + ), + ] + + endpoint_url = f"http://{started_cluster.minio_ip}:{started_cluster.minio_port}" + aws_access_key_id = "minio" + aws_secret_access_key = "minio123" + + storage_options = { + "AWS_ENDPOINT_URL": endpoint_url, + "AWS_ACCESS_KEY_ID": aws_access_key_id, + "AWS_SECRET_ACCESS_KEY": aws_secret_access_key, + "AWS_ALLOW_HTTP": "true", + "AWS_S3_ALLOW_UNSAFE_RENAME": "true", + } + path = f"s3://root/testtable" + table = pa.Table.from_arrays(data, schema=schema) + + write_deltalake(path, table, storage_options=storage_options) + + assert "1\n2\n3\n" in node.query( + f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + assert ( + "('123 Elm St','Springfield','IL')\n('456 Maple St','Shelbyville','IL')\n('789 Oak St','Ogdenville','IL')" + in node.query( + f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + ) + assert ( + "{'key1':'value1','key2':'value2'}\n{'key1':'value3','key2':'value4'}\n{'key1':'value5','key2':'value6'}" + in node.query( + f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + ) From 6e584dd541527e9171e245b5a95bd6c5bc388deb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 26 Aug 2024 11:33:08 +0200 Subject: [PATCH 41/41] Fix test --- tests/integration/test_storage_delta/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d000875d8d1..0aa846478ea 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -795,6 +795,7 @@ def test_complex_types(started_cluster): endpoint_url = f"http://{started_cluster.minio_ip}:{started_cluster.minio_port}" aws_access_key_id = "minio" aws_secret_access_key = "minio123" + table_name = randomize_table_name("test_complex_types") storage_options = { "AWS_ENDPOINT_URL": endpoint_url, @@ -803,23 +804,23 @@ def test_complex_types(started_cluster): "AWS_ALLOW_HTTP": "true", "AWS_S3_ALLOW_UNSAFE_RENAME": "true", } - path = f"s3://root/testtable" + path = f"s3://root/{table_name}" table = pa.Table.from_arrays(data, schema=schema) write_deltalake(path, table, storage_options=storage_options) assert "1\n2\n3\n" in node.query( - f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) assert ( "('123 Elm St','Springfield','IL')\n('456 Maple St','Shelbyville','IL')\n('789 Oak St','Ogdenville','IL')" in node.query( - f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) ) assert ( "{'key1':'value1','key2':'value2'}\n{'key1':'value3','key2':'value4'}\n{'key1':'value5','key2':'value6'}" in node.query( - f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) )