From cc5456c649a4bd1f95321f4320fadfa382236d1b Mon Sep 17 00:00:00 2001 From: cangyin Date: Sat, 6 Apr 2024 19:38:37 +0000 Subject: [PATCH 001/409] 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 002/409] 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 003/409] 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 004/409] 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 3fb45ff1762867cbeb53e31cd4492dda5cf8dbb7 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 14 May 2024 19:17:56 +0200 Subject: [PATCH 005/409] Add setting and implementation --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/SchemaInferenceUtils.cpp | 44 ++++++++++++++++++- .../0_stateless/03150_infer_type_variant.sql | 4 ++ 5 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03150_infer_type_variant.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a03..0c0614550e5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,6 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ + M(Bool, input_format_json_infer_variant_from_multitype_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 43ccee173f0..e027c693094 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,6 +137,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; + format_settings.json.infer_variant_from_multitype_array = settings.input_format_json_infer_variant_from_multitype_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d5fedf99adb..6cac41bd63e 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -223,6 +223,7 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; + bool infer_variant_from_multitype_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 02c0aa6dd77..fc1a26f9b2f 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -307,6 +308,22 @@ namespace type_indexes.erase(TypeIndex::UInt64); } + /// if setting input_format_json_infer_variant_from_multitype_array is true + /// and nested types are not equal then we convert to type variant. + void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + auto variant_type = std::make_shared(data_types); + /// replace separate types with a single variant type + data_types.clear(); + type_indexes.clear(); + data_types.push_back(variant_type); + type_indexes.insert(TypeIndex::Variant); + + // push it back again + data_types.push_back(variant_type); + type_indexes.insert(TypeIndex::Variant); + } + /// If we have only Date and DateTime types, convert Date to DateTime, /// otherwise, convert all Date and DateTime to String. void transformDatesAndDateTimes(DataTypes & data_types, TypeIndexesSet & type_indexes) @@ -649,6 +666,12 @@ namespace /// Check settings specific for JSON formats. + if (settings.json.infer_variant_from_multitype_array) + { + transformVariant(data_types, type_indexes); + return; + } + /// Convert numbers inferred from strings back to strings if needed. if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings) transformJSONNumbersBackToString(data_types, settings, type_indexes, json_info); @@ -677,6 +700,12 @@ namespace if constexpr (!is_json) return; + if (settings.json.infer_variant_from_multitype_array) + { + transformVariant(data_types, type_indexes); + return; + } + /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); @@ -822,7 +851,6 @@ namespace if (checkIfTypesAreEqual(nested_types_copy)) return std::make_shared(nested_types_copy.back()); - return std::make_shared(nested_types); } else @@ -1482,6 +1510,20 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) return nested_type ? std::make_shared(nested_type) : nullptr; } + if (which.isVariant()) + { + const auto * variant_type = assert_cast(type.get()); + DataTypes nested_types; + for (const auto & nested_type: variant_type->getVariants()) + { + /// unlike tuple or array, here we do not want to make any of the variants nullable + /// so we do not call makeNullableRecursively + nested_types.push_back(nested_type); + } + + return std::make_shared(nested_types); + } + if (which.isTuple()) { const auto * tuple_type = assert_cast(type.get()); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql new file mode 100644 index 00000000000..ac544d04f6e --- /dev/null +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -0,0 +1,4 @@ +SET input_format_json_infer_variant_from_multitype_array=1; +SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From 467366af990215e11b4b0309b90b3e6d9ebca5fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 14 May 2024 21:55:56 +0200 Subject: [PATCH 006/409] Fix unimplemented serialization error and update reference file --- src/Formats/SchemaInferenceUtils.cpp | 13 ++++++++--- .../03150_infer_type_variant.reference | 22 +++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03150_infer_type_variant.reference diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index fc1a26f9b2f..d0d29892dec 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -319,7 +319,7 @@ namespace data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); - // push it back again + // make the second type variant as well data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); } @@ -669,7 +669,6 @@ namespace if (settings.json.infer_variant_from_multitype_array) { transformVariant(data_types, type_indexes); - return; } /// Convert numbers inferred from strings back to strings if needed. @@ -703,7 +702,6 @@ namespace if (settings.json.infer_variant_from_multitype_array) { transformVariant(data_types, type_indexes); - return; } /// Convert JSON tuples with same nested types to arrays. @@ -1440,6 +1438,15 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F return; } + + if (const auto * variant_type = typeid_cast(data_type.get())) + { + auto nested_types = variant_type->getVariants(); + for (auto & nested_type : nested_types) + transformFinalInferredJSONTypeIfNeededImpl(nested_type, settings, json_info, remain_nothing_types); + data_type = std::make_shared(nested_types); + return; + } } void transformFinalInferredJSONTypeIfNeeded(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference new file mode 100644 index 00000000000..ffb4209eadb --- /dev/null +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -0,0 +1,22 @@ + ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ + │ │… a Int64))) │ + └──────────────────┴─────────────────────────────────────┘ + ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ x ┃ toTypeName(x) ┃ + ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 42 │ Variant(Int64, String) │ + ├───────┼────────────────────────┤ +2. │ Hello │ Variant(Int64, String) │ + └───────┴────────────────────────┘ + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ x ┃ toTypeName(x) ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ [1,2,3] │ Variant(Array(Int64), Tuple(…│ + │ │… a Int64)) │ + ├─────────┼──────────────────────────────┤ +2. │ (42) │ Variant(Array(Int64), Tuple(…│ + │ │… a Int64)) │ + └─────────┴──────────────────────────────┘ From 4066c6bc548979703f45ba264437f5966c403d6a Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 02:13:53 +0000 Subject: [PATCH 007/409] Update setting name --- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 6 +++--- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0c0614550e5..ffc337b674f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,7 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_infer_variant_from_multitype_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ + M(Bool, input_format_json_infer_variant_from_multi_type_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index e027c693094..792ac08a5df 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,7 +137,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; - format_settings.json.infer_variant_from_multitype_array = settings.input_format_json_infer_variant_from_multitype_array; + format_settings.json.infer_variant_from_multi_type_array = settings.input_format_json_infer_variant_from_multi_type_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6cac41bd63e..d2c75872326 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -223,7 +223,7 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; - bool infer_variant_from_multitype_array = false; + bool infer_variant_from_multi_type_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index d0d29892dec..f693916c584 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -308,7 +308,7 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting input_format_json_infer_variant_from_multitype_array is true + /// if setting input_format_json_infer_variant_from_multi_type_array is true /// and nested types are not equal then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { @@ -666,7 +666,7 @@ namespace /// Check settings specific for JSON formats. - if (settings.json.infer_variant_from_multitype_array) + if (settings.json.infer_variant_from_multi_type_array) { transformVariant(data_types, type_indexes); } @@ -699,7 +699,7 @@ namespace if constexpr (!is_json) return; - if (settings.json.infer_variant_from_multitype_array) + if (settings.json.infer_variant_from_multi_type_array) { transformVariant(data_types, type_indexes); } diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index ac544d04f6e..2ea849248f7 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multitype_array=1; +SET input_format_json_infer_variant_from_multi_type_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From 2762cf86d2ba3f5c1ac86040b6ef484feb40837b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 14:19:17 +0200 Subject: [PATCH 008/409] fix test file --- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 2ea849248f7..ac544d04f6e 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multi_type_array=1; +SET input_format_json_infer_variant_from_multitype_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From 4800aa6a6cdf5a8431fcc3d6fd96672590da0fff Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 14:21:52 +0200 Subject: [PATCH 009/409] rename setting in test file --- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index ac544d04f6e..2ea849248f7 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multitype_array=1; +SET input_format_json_infer_variant_from_multi_type_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From fbf34519a5e72bb03c57ee6bc0feea1adddcb309 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 16:03:23 +0200 Subject: [PATCH 010/409] Add setting to SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ece48620618..0665d1d6ca6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,7 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, + {"input_format_json_infer_variant_from_multi_type_array", 0, 0, "Allows inference of variant type if columns/arrays have multiple types."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 777e0b313a48975fa51d645fb09a4f6ebfac1d1e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 18:51:29 +0200 Subject: [PATCH 011/409] Update 03150_infer_type_variant.sql Fix failing test --- tests/queries/0_stateless/03150_infer_type_variant.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 2ea849248f7..3253ddfe179 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ SET input_format_json_infer_variant_from_multi_type_array=1; -SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); -SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); -SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); +SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}') FORMAT Pretty; +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}') FORMAT Pretty; +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}') FORMAT Pretty; From dd8d5c46c4678f570b9357dbdd912bcd6f4a267e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 18:55:34 +0200 Subject: [PATCH 012/409] Fix style --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index f693916c584..2cbb680af97 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -319,7 +319,7 @@ namespace data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); - // make the second type variant as well + /// make the second type variant as well data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); } From 04800f596c4471d10e15c40a533c539c6b549b06 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 24 May 2024 21:20:20 +0200 Subject: [PATCH 013/409] Incorporate review changes --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 60 +++++++++++++------ .../03150_infer_type_variant.reference | 29 +++++---- .../0_stateless/03150_infer_type_variant.sql | 3 +- 7 files changed, 67 insertions(+), 33 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ffc337b674f..be7564794e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,7 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_infer_variant_from_multi_type_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ + M(Bool, input_format_try_infer_variants, false, "Try to infer the Variant type in text formats when there is more than one possible type for column/array elements", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 0665d1d6ca6..75fcb538b2b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,7 +93,7 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, - {"input_format_json_infer_variant_from_multi_type_array", 0, 0, "Allows inference of variant type if columns/arrays have multiple types."}, + {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 792ac08a5df..2854802453e 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,7 +137,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; - format_settings.json.infer_variant_from_multi_type_array = settings.input_format_json_infer_variant_from_multi_type_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; @@ -266,6 +265,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); format_settings.date_time_overflow_behavior = settings.date_time_overflow_behavior; + format_settings.try_infer_variant = settings.input_format_try_infer_variants; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 927a7e691d8..a2a9e75bd44 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -36,6 +36,7 @@ struct FormatSettings bool decimal_trailing_zeros = false; bool defaults_for_omitted_fields = true; bool is_writing_to_terminal = false; + bool try_infer_variant = false; bool seekable_read = true; UInt64 max_rows_to_read_for_schema_inference = 25000; @@ -223,7 +224,6 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; - bool infer_variant_from_multi_type_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 2cbb680af97..298127cad68 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -239,6 +239,16 @@ namespace return true; } + bool checkIfTypesContainVariant(const DataTypes & types) + { + for (size_t i = 0; i < types.size(); ++i) + { + if (isVariant(types[i])) + return true; + } + return false; + } + void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) { type_indexes.clear(); @@ -308,20 +318,31 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting input_format_json_infer_variant_from_multi_type_array is true + /// if setting try_infer_variant is true /// and nested types are not equal then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { + auto typesAreEqual = checkIfTypesAreEqual(data_types); + auto typesContainVariant = checkIfTypesContainVariant(data_types); + if (typesAreEqual || typesContainVariant) + return; + + DataTypes new_data_types; + TypeIndexesSet new_type_indexes; + auto variant_type = std::make_shared(data_types); - /// replace separate types with a single variant type + size_t i = 0; + while (i != data_types.size()) + { + new_data_types.push_back(variant_type); + new_type_indexes.insert(TypeIndex::Variant); + i++; + } + data_types.clear(); type_indexes.clear(); - data_types.push_back(variant_type); - type_indexes.insert(TypeIndex::Variant); - - /// make the second type variant as well - data_types.push_back(variant_type); - type_indexes.insert(TypeIndex::Variant); + data_types = new_data_types; + type_indexes = new_type_indexes; } /// If we have only Date and DateTime types, convert Date to DateTime, @@ -661,16 +682,14 @@ namespace if (settings.try_infer_dates || settings.try_infer_datetimes) transformDatesAndDateTimes(data_types, type_indexes); + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + if constexpr (!is_json) return; /// Check settings specific for JSON formats. - if (settings.json.infer_variant_from_multi_type_array) - { - transformVariant(data_types, type_indexes); - } - /// Convert numbers inferred from strings back to strings if needed. if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings) transformJSONNumbersBackToString(data_types, settings, type_indexes, json_info); @@ -685,6 +704,10 @@ namespace if (settings.json.try_infer_objects_as_tuples) mergeJSONPaths(data_types, type_indexes, settings, json_info); + + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + }; auto transform_complex_types = [&](DataTypes & data_types, TypeIndexesSet & type_indexes) @@ -696,14 +719,12 @@ namespace /// If there is at least one non Nothing type, change all Nothing types to it. transformNothingComplexTypes(data_types, type_indexes); + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + if constexpr (!is_json) return; - if (settings.json.infer_variant_from_multi_type_array) - { - transformVariant(data_types, type_indexes); - } - /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); @@ -715,6 +736,9 @@ namespace if (json_info && json_info->allow_merging_named_tuples) mergeNamedTuples(data_types, type_indexes, settings, json_info); + + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); }; transformTypesRecursively(types, transform_simple_types, transform_complex_types); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index ffb4209eadb..f3f53057845 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -2,7 +2,7 @@ ┃ arr ┃ toTypeName(arr) ┃ ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ 1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ - │ │… a Int64))) │ + │ │… a Nullable(Int64)))) │ └──────────────────┴─────────────────────────────────────┘ ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ ┃ x ┃ toTypeName(x) ┃ @@ -11,12 +11,21 @@ ├───────┼────────────────────────┤ 2. │ Hello │ Variant(Int64, String) │ └───────┴────────────────────────┘ - ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ x ┃ toTypeName(x) ┃ - ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ [1,2,3] │ Variant(Array(Int64), Tuple(…│ - │ │… a Int64)) │ - ├─────────┼──────────────────────────────┤ -2. │ (42) │ Variant(Array(Int64), Tuple(…│ - │ │… a Int64)) │ - └─────────┴──────────────────────────────┘ + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ x ┃ toTypeName(x) ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple(…│ + │ │… a Nullable(Int64))) │ + ├─────────┼────────────────────────────────────────┤ +2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple(…│ + │ │… a Nullable(Int64))) │ + └─────────┴────────────────────────────────────────┘ + ┏━━━━┳━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ + ┡━━━━╇━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━┩ +1. │ 1 │ Nullable(Int64) │ Hello World! │ Nullable(String) │ + ├────┼─────────────────┼──────────────┼──────────────────┤ +2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Nullable(String) │ + ├────┼─────────────────┼──────────────┼──────────────────┤ +3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Nullable(String) │ + └────┴─────────────────┴──────────────┴──────────────────┘ diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 3253ddfe179..45126ccd471 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,5 @@ -SET input_format_json_infer_variant_from_multi_type_array=1; +SET input_format_try_infer_variants=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}') FORMAT Pretty; SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}') FORMAT Pretty; SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}') FORMAT Pretty; +SELECT c1, toTypeName(c1), c2, toTypeName(c2) FROM format('CSV', '1,Hello World!\n2,"[1,2,3]"\n3,"2020-01-01"\n') FORMAT Pretty; \ No newline at end of file From 655262d1a1b21d85f4fbe284e0835065bcca379b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 5 Jun 2024 14:34:17 +0200 Subject: [PATCH 014/409] Fix issue with nullables --- src/Formats/SchemaInferenceUtils.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 298127cad68..43120cb7b22 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1547,11 +1547,13 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) DataTypes nested_types; for (const auto & nested_type: variant_type->getVariants()) { - /// unlike tuple or array, here we do not want to make any of the variants nullable - /// so we do not call makeNullableRecursively - nested_types.push_back(nested_type); + auto is_low_cardinality = nested_type->lowCardinality(); + auto has_sub_types = nested_type->haveSubtypes(); + if (!is_low_cardinality && has_sub_types) + nested_types.push_back(makeNullableRecursively(nested_type)); + else + nested_types.push_back(nested_type); } - return std::make_shared(nested_types); } From 418fc7f4438abd25eae4928f36ff0c3fef2395f8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 10:02:19 +0200 Subject: [PATCH 015/409] Fix incorrect inference for other formats --- src/Formats/SchemaInferenceUtils.cpp | 29 +++++++++-- .../03150_infer_type_variant.reference | 48 +++++++++---------- 2 files changed, 49 insertions(+), 28 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 43120cb7b22..a8b5d4343f5 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -318,19 +318,40 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting try_infer_variant is true - /// and nested types are not equal then we convert to type variant. + /// if setting 'try_infer_variant' is true then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { auto typesAreEqual = checkIfTypesAreEqual(data_types); auto typesContainVariant = checkIfTypesContainVariant(data_types); - if (typesAreEqual || typesContainVariant) + if (typesAreEqual) return; DataTypes new_data_types; TypeIndexesSet new_type_indexes; + std::shared_ptr variant_type; + + /// extract the nested types of variant and make a new variant with the nested types and the other type. + /// eg. Type 1: variant, Type 2: Date -> variant. + if (typesContainVariant) + { + DataTypes extracted_types; + for (size_t i=0; i(data_types[i].get())) + extracted_types = variant->getVariants(); + } + else + extracted_types.push_back(data_types[i]); + } + variant_type = std::make_shared(extracted_types); + } + else + { + variant_type = std::make_shared(data_types); + } - auto variant_type = std::make_shared(data_types); size_t i = 0; while (i != data_types.size()) { diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index f3f53057845..a5f56cb3618 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -1,9 +1,9 @@ - ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ arr ┃ toTypeName(arr) ┃ - ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ - │ │… a Nullable(Int64)))) │ - └──────────────────┴─────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple( + a Nullable(Int64)))) │ + └──────────────────┴─────────────────────────────────────────────────────────────┘ ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ ┃ x ┃ toTypeName(x) ┃ ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ @@ -11,21 +11,21 @@ ├───────┼────────────────────────┤ 2. │ Hello │ Variant(Int64, String) │ └───────┴────────────────────────┘ - ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ x ┃ toTypeName(x) ┃ - ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple(…│ - │ │… a Nullable(Int64))) │ - ├─────────┼────────────────────────────────────────┤ -2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple(…│ - │ │… a Nullable(Int64))) │ - └─────────┴────────────────────────────────────────┘ - ┏━━━━┳━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━┓ - ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ - ┡━━━━╇━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━┩ -1. │ 1 │ Nullable(Int64) │ Hello World! │ Nullable(String) │ - ├────┼─────────────────┼──────────────┼──────────────────┤ -2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Nullable(String) │ - ├────┼─────────────────┼──────────────┼──────────────────┤ -3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Nullable(String) │ - └────┴─────────────────┴──────────────┴──────────────────┘ + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ x ┃ toTypeName(x) ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple( + a Nullable(Int64))) │ + ├─────────┼───────────────────────────────────────────────────────────────┤ +2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple( + a Nullable(Int64))) │ + └─────────┴───────────────────────────────────────────────────────────────┘ + ┏━━━━┳━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ + ┡━━━━╇━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 1 │ Nullable(Int64) │ Hello World! │ Variant(Array(Nullable(Int64)), String) │ + ├────┼─────────────────┼──────────────┼─────────────────────────────────────────┤ +2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Variant(Array(Nullable(Int64)), String) │ + ├────┼─────────────────┼──────────────┼─────────────────────────────────────────┤ +3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Variant(Array(Nullable(Int64)), String) │ + └────┴─────────────────┴──────────────┴─────────────────────────────────────────┘ From 9cf11a210f07110676b373b864ea098583d87ff6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 11 Jun 2024 11:11:06 +0200 Subject: [PATCH 016/409] Review changes --- src/Core/SettingsChangesHistory.h | 3 +- src/Formats/SchemaInferenceUtils.cpp | 73 ++++++------------- .../03150_infer_type_variant.reference | 24 +++--- 3 files changed, 35 insertions(+), 65 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3f743ef42bf..661ecc607ba 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, @@ -103,8 +104,6 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, - {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index a8b5d4343f5..b7c71a95b29 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -239,16 +239,6 @@ namespace return true; } - bool checkIfTypesContainVariant(const DataTypes & types) - { - for (size_t i = 0; i < types.size(); ++i) - { - if (isVariant(types[i])) - return true; - } - return false; - } - void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) { type_indexes.clear(); @@ -321,49 +311,28 @@ namespace /// if setting 'try_infer_variant' is true then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { - auto typesAreEqual = checkIfTypesAreEqual(data_types); - auto typesContainVariant = checkIfTypesContainVariant(data_types); - if (typesAreEqual) + if (checkIfTypesAreEqual(data_types)) return; - DataTypes new_data_types; - TypeIndexesSet new_type_indexes; - std::shared_ptr variant_type; - - /// extract the nested types of variant and make a new variant with the nested types and the other type. - /// eg. Type 1: variant, Type 2: Date -> variant. - if (typesContainVariant) + DataTypes variant_types; + for (const auto & type : data_types) { - DataTypes extracted_types; - for (size_t i=0; i(type.get())) { - if (isVariant(data_types[i])) - { - if (const auto * variant = typeid_cast(data_types[i].get())) - extracted_types = variant->getVariants(); - } - else - extracted_types.push_back(data_types[i]); + const auto & current_variants = variant_type->getVariants(); + variant_types.insert(variant_types.end(), current_variants.begin(), current_variants.end()); + } + else + { + variant_types.push_back(type); } - variant_type = std::make_shared(extracted_types); - } - else - { - variant_type = std::make_shared(data_types); } - size_t i = 0; - while (i != data_types.size()) - { - new_data_types.push_back(variant_type); - new_type_indexes.insert(TypeIndex::Variant); - i++; - } + auto variant_type = std::make_shared(variant_types); - data_types.clear(); - type_indexes.clear(); - data_types = new_data_types; - type_indexes = new_type_indexes; + for (auto & type : data_types) + type = variant_type; + type_indexes = {TypeIndex::Variant}; } /// If we have only Date and DateTime types, convert Date to DateTime, @@ -703,11 +672,12 @@ namespace if (settings.try_infer_dates || settings.try_infer_datetimes) transformDatesAndDateTimes(data_types, type_indexes); - if (settings.try_infer_variant) - transformVariant(data_types, type_indexes); - if constexpr (!is_json) + { + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); return; + } /// Check settings specific for JSON formats. @@ -740,11 +710,12 @@ namespace /// If there is at least one non Nothing type, change all Nothing types to it. transformNothingComplexTypes(data_types, type_indexes); - if (settings.try_infer_variant) - transformVariant(data_types, type_indexes); - if constexpr (!is_json) + { + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); return; + } /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index a5f56cb3618..a43fa1e1227 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -1,16 +1,16 @@ - ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ arr ┃ toTypeName(arr) ┃ - ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple( + ┏━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ ['1','Hello',(32)] │ Array(Variant(String, Tuple( a Nullable(Int64)))) │ - └──────────────────┴─────────────────────────────────────────────────────────────┘ - ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ x ┃ toTypeName(x) ┃ - ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ 42 │ Variant(Int64, String) │ - ├───────┼────────────────────────┤ -2. │ Hello │ Variant(Int64, String) │ - └───────┴────────────────────────┘ + └────────────────────┴──────────────────────────────────────────────────────┘ + ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━┓ + ┃ x ┃ toTypeName(x) ┃ + ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━┩ +1. │ 42 │ Nullable(String) │ + ├───────┼──────────────────┤ +2. │ Hello │ Nullable(String) │ + └───────┴──────────────────┘ ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ ┃ x ┃ toTypeName(x) ┃ ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ From 548c90901020317669a10d191a0b6f8a7d0a0511 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 11 Jun 2024 12:14:36 +0200 Subject: [PATCH 017/409] Add documentation --- docs/en/operations/settings/settings-formats.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 6aae1ea62e5..8bbb469547b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -194,6 +194,17 @@ If enabled, ClickHouse will try to infer type `DateTime64` from string fields in Enabled by default. +## input_format_try_infer_variants {#input_format_try_infer_variants} + +If enabled, ClickHouse will try to infer type [`Variant`](../../sql-reference/data-types/variant.md) in schema inference for text formats when there is more than one possible type for column/array elements. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. + ## date_time_input_format {#date_time_input_format} Allows choosing a parser of the text representation of date and time. From bad5e27bbffa9c1f6727a0416edcb135dadcc1fe Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 11 Jun 2024 13:32:34 +0200 Subject: [PATCH 018/409] Update src/Formats/SchemaInferenceUtils.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Formats/SchemaInferenceUtils.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 0ac8b32f8aa..240830013c6 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1539,9 +1539,7 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) DataTypes nested_types; for (const auto & nested_type: variant_type->getVariants()) { - auto is_low_cardinality = nested_type->lowCardinality(); - auto has_sub_types = nested_type->haveSubtypes(); - if (!is_low_cardinality && has_sub_types) + if (!nested_type->lowCardinality() && nested_type->haveSubtypes()) nested_types.push_back(makeNullableRecursively(nested_type)); else nested_types.push_back(nested_type); From 384aa9feb90bbf95c5bc0e5498af4aca769c2531 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jun 2024 13:51:20 +0200 Subject: [PATCH 019/409] Move setting to 24.7 changes --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 1ab7dc69f60..deaeba2a7de 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,6 +86,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { + {"24.7", {{"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, + }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, @@ -97,7 +99,6 @@ static const std::map Date: Wed, 3 Jul 2024 09:33:57 +0200 Subject: [PATCH 020/409] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7719fe1e837..828031f4c23 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list Date: Sat, 13 Jul 2024 14:06:24 +0000 Subject: [PATCH 021/409] Add local storage --- .../ObjectStorage/Local/Configuration.cpp | 75 +++++++++ .../ObjectStorage/Local/Configuration.h | 55 +++++++ .../test_local_storage/__init__.py | 0 tests/integration/test_local_storage/test.py | 153 ++++++++++++++++++ 4 files changed, 283 insertions(+) create mode 100644 src/Storages/ObjectStorage/Local/Configuration.cpp create mode 100644 src/Storages/ObjectStorage/Local/Configuration.h create mode 100644 tests/integration/test_local_storage/__init__.py create mode 100644 tests/integration/test_local_storage/test.py diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp new file mode 100644 index 00000000000..5b66dd2470b --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -0,0 +1,75 @@ +#include + +#include +#include +#include +#include "Common/NamedCollections/NamedCollections.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int LOGICAL_ERROR; +} + +void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) +{ + path = collection.get("path"); + format = collection.getOrDefault("format", "auto"); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); +} + + +void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + const size_t max_args_num = with_structure ? 4 : 3; + if (args.empty() || args.size() > max_args_num) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num); + } + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + path = checkAndGetLiteralArgument(args[0], "path"); + + if (args.size() > 1) + { + format = checkAndGetLiteralArgument(args[1], "format_name"); + } + + if (with_structure) + { + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + } + if (args.size() > 3) + { + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + } + } + else if (args.size() > 2) + { + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + } +} + +StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.engine_file_skip_empty_files, + .list_object_keys_size = 0, + .throw_on_zero_files_match = false, + .ignore_non_existent_file = false}; +} + +} diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h new file mode 100644 index 00000000000..a305fc43825 --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include "Disks/ObjectStorages/Local/LocalObjectStorage.h" +#include "config.h" + +#if USE_AWS_S3 +# include +# include + +namespace DB +{ + +class StorageLocalConfiguration : public StorageObjectStorage::Configuration +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "local"; + + StorageLocalConfiguration() = default; + StorageLocalConfiguration(const StorageLocalConfiguration & other) = default; + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return "Local"; } + + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + void setPaths(const Paths & paths_) override { paths = paths_; } + + String getNamespace() const override { return ""; } + String getDataSourceDescription() const override { return ""; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + void check(ContextPtr) const override { } + void validateNamespace(const String &) const override { } + ConfigurationPtr clone() override { return std::make_shared(*this); } + bool isStaticConfiguration() const override { return true; } + + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + + void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } + +private: + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + Path path; + Paths paths; +}; + +} + +#endif diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py new file mode 100644 index 00000000000..96d116ec6a2 --- /dev/null +++ b/tests/integration/test_local_storage/test.py @@ -0,0 +1,153 @@ +import logging +import os +import random +import string + +import pytest + +from helpers.cluster import ClickHouseCluster +from azure.storage.blob import BlobServiceClient +from test_storage_azure_blob_storage.test import azure_query + +NODE_NAME = "node" + + +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/disk_storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + + object_storage + azure_blob_storage + plain_rewritable + http://azurite1:{port}/devstoreaccount1 + cont + true + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + 100000 + 100000 + 10 + 10 + + + + + +
+ blob_storage_disk +
+
+
+
+
+
+""" + ) + return path + + +insert_values = [ + "(0,'data'),(1,'data')", + ",".join( + f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" + for i in range(10) + ), +] + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) + cluster.add_instance( + NODE_NAME, + main_configs=[ + path, + ], + with_azurite=True, + stay_alive=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_insert_select(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + azure_query( + node, + """ + CREATE TABLE test_{} ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='blob_storage_policy' + """.format( + index + ), + ) + + azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_restart_server(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + node.restart_clickhouse() + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_drop_table(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) + + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + assert len(list(container_client.list_blobs())) == 0 From f54a4b073a3fd1c0d31b26c3085eedabb667687a Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 13 Jul 2024 14:07:36 +0000 Subject: [PATCH 022/409] Add local table function --- .../registerStorageObjectStorage.cpp | 24 +++++++++++++++++-- .../TableFunctionObjectStorage.h | 22 +++++++++++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index bf595b2f5d4..04c3417948c 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,9 +1,10 @@ +#include #include -#include #include +#include +#include #include #include -#include namespace DB { @@ -138,8 +139,27 @@ void registerStorageHDFS(StorageFactory & factory) } #endif +void registerStorageLocal(StorageFactory & factory) +{ + factory.registerStorage( + "Local", + [=](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, + }); +} + + void registerStorageObjectStorage(StorageFactory & factory) { + registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From d2eded16aaa62fe66e93bac171c25a1f031b3881 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 15 Jul 2024 10:53:22 +0000 Subject: [PATCH 023/409] Debug commit --- src/CMakeLists.txt | 1 + .../Local/LocalObjectStorage.cpp | 12 ++-- src/IO/ReadHelpers.cpp | 10 ++- .../Executors/PullingPipelineExecutor.cpp | 13 ++++ .../ObjectStorage/Local/Configuration.cpp | 2 + .../ObjectStorage/Local/Configuration.h | 20 +++--- .../ObjectStorage/StorageObjectStorage.cpp | 12 ++++ .../StorageObjectStorageSource.cpp | 62 ++++++++++++++++--- .../TableFunctionObjectStorage.cpp | 15 +++-- 9 files changed, 116 insertions(+), 31 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..dc32687e058 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -110,6 +110,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage) add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) +add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index a247d86ddce..6c21de96371 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,15 +1,17 @@ #include -#include -#include -#include +#include +#include +#include +#include #include #include -#include #include #include +#include +#include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..ddc2e912fb3 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,6 +80,7 @@ UUID parseUUID(std::span src) return uuid; } + void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -88,12 +89,15 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); - throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); + throw Exception( + ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, + "Cannot parse input: expected {} {}", + out.str(), + buf.buffer().end() - buf.position()); } - bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..a4db27d3f37 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,19 +42,32 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { + LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); + if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } + LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); + if (!executor->checkTimeLimitSoft()) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); + + if (!executor->executeStep(&has_data_flag)) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); + + chunk = pulling_format->getChunk(); + + LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); + return true; } diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 5b66dd2470b..3abdb4d530b 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -20,6 +20,7 @@ void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & coll format = collection.getOrDefault("format", "auto"); compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); + paths = {path}; } @@ -56,6 +57,7 @@ void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool wi { compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } + paths = {path}; } StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index a305fc43825..11165fcd574 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -2,11 +2,13 @@ #include #include "Disks/ObjectStorages/Local/LocalObjectStorage.h" -#include "config.h" -#if USE_AWS_S3 -# include -# include +#include + +#include + + +namespace fs = std::filesystem; namespace DB { @@ -34,12 +36,12 @@ public: String getDataSourceDescription() const override { return ""; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; - void check(ContextPtr) const override { } - void validateNamespace(const String &) const override { } ConfigurationPtr clone() override { return std::make_shared(*this); } - bool isStaticConfiguration() const override { return true; } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override + { + return std::make_shared(fs::path{path}.parent_path()); + } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } @@ -51,5 +53,3 @@ private: }; } - -#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 683473006e3..8c5b4f71d4d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -124,6 +124,12 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { + LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } } std::string getName() const override { return name; } @@ -422,6 +428,12 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c DEFAULT_SCHEMA_CACHE_ELEMENTS)); return schema_cache; } + else if (storage_type_name == "local") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_local", DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a9a7e062076..54ddaec0140 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,17 +1,18 @@ #include "StorageObjectStorageSource.h" -#include #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include "Common/logger_useful.h" #include namespace fs = std::filesystem; @@ -69,6 +70,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { + LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -131,6 +133,7 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) { auto keys = configuration->getPaths(); @@ -142,6 +145,19 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } + + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); + for (auto && key : copy_configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); + } + + iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -170,8 +186,11 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); + while (true) { + LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); + if (isCancelled() || !reader) { if (reader) @@ -179,10 +198,15 @@ Chunk StorageObjectStorageSource::generate() break; } + LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); + Chunk chunk; if (reader->pull(chunk)) { + LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); + UInt64 num_rows = chunk.getNumRows(); + LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -230,6 +254,9 @@ Chunk StorageObjectStorageSource::generate() return chunk; } + LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -300,6 +327,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); + LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); + QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -324,11 +353,17 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; + LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); + + std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; + LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); + + if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -341,6 +376,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { + LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); + CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -625,11 +662,18 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { + LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); + for (auto && key : keys) + { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); + } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? + for (auto && key : keys) { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..d156afb81a1 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,12 +12,13 @@ #include -#include #include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -203,6 +204,11 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif + factory.registerFunction>( + {.documentation + = {.description = R"(The table function can be used to read the data stored locally.)", + .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, + .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -223,4 +229,5 @@ template class TableFunctionObjectStorage; #endif +template class TableFunctionObjectStorage; } From 0bfe345a460766e8113934801609230410db1836 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 17 Jul 2024 09:24:33 +0000 Subject: [PATCH 024/409] Debug code --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 7 ++++--- src/Storages/ObjectStorage/Local/Configuration.h | 5 +---- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 7 +++++++ 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 6c21de96371..dfbc1a77d3c 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -50,10 +50,11 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { + LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); + auto from_file_storage = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index 11165fcd574..ba4de63ac47 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -38,10 +38,7 @@ public: ConfigurationPtr clone() override { return std::make_shared(*this); } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override - { - return std::make_shared(fs::path{path}.parent_path()); - } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared("/"); } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 54ddaec0140..715f93c7f7e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -388,7 +388,12 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); + LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); + auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); + std::string answer(1000, ' '); + size_t read_bytes = new_read_buf->read(answer.data(), 1000); + LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -471,6 +476,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Read"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); From 84559ab31cac8d994fbccf9085884c1c2ea8f4b8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 10:06:06 +0000 Subject: [PATCH 025/409] Debug code --- .../Local/LocalObjectStorage.cpp | 100 +++++++++--------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 75 ++++++++----- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageSource.cpp | 36 ++++--- 4 files changed, 120 insertions(+), 93 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 3704e4bb672..607ce41dd0f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -58,27 +59,28 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL std::string answer(1000, ' '); size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase 00"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; + LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, @@ -86,56 +88,58 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL } case RemoteFSReadMethod::threadpool: { - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); + // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); auto impl = std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - auto impl3 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl3 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + + // auto async_reader = std::make_unique( + // std::move(impl3), + // reader2, + // read_settings, + // global_context->getAsyncReadCounters(), + // global_context->getFilesystemReadPrefetchesLog()); + + // answer = std::string(1000, ' '); + // read_bytes = async_reader->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - std::move(impl3), + return std::make_unique( + std::move(impl), reader, read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); - - answer = std::string(1000, ' '); - read_bytes = async_reader->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - "Read bytes: {}, string: {}", - read_bytes, - answer.substr(0, read_bytes)); - - // reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), reader, read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); } } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 267c3eaaea4..351f167b6b9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,21 +200,41 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; + // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( + // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr + // { + // return std::make_unique( + // client.get(), + // uri.bucket, + // object_.remote_path, + // uri.version_id, + // settings_ptr->request_settings, + // disk_read_settings, + // /* use_external_buffer */ true, + // /* offset */ 0, + // /* read_until_position */ 0, + // restricted_seek); + // }; + + switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ false); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), @@ -224,17 +244,24 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: - { + case RemoteFSReadMethod::threadpool: { + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto impl = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */true); - - auto impl2 = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", @@ -242,12 +269,6 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( std::move(impl), reader, disk_read_settings, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 818ce055c77..7a13ada224c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -157,7 +157,7 @@ public: ContextPtr local_context, bool with_table_structure); - /// Storage type: s3, hdfs, azure. + /// Storage type: s3, hdfs, azure, local. virtual std::string getTypeName() const = 0; /// Engine name: S3, HDFS, Azure. virtual std::string getEngineName() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index f664bed0204..8b800cbc343 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,6 +461,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); + read_settings.remote_fs_method = RemoteFSReadMethod::read; + LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); read_settings.enable_filesystem_cache = false; @@ -477,28 +479,28 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - if (use_prefetch) - { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + // if (use_prefetch) + // { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); - auto async_reader = object_storage->readObjects( - StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + auto async_reader + = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - return async_reader; - } - else - { - /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); + return async_reader; + // } + // else + // { + // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - } + // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + // } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) From 8fad2860251abf3fc71b525522c7ce51bd4687aa Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 15:03:12 +0000 Subject: [PATCH 026/409] Remove asyncronous processing for local storage --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 5 +- .../Local/LocalObjectStorage.cpp | 96 ++----------------- .../ObjectStorage/Local/Configuration.cpp | 2 +- .../StorageObjectStorageSource.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 6 files changed, 20 insertions(+), 91 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 92b1c750a55..10b869e58f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,6 +114,7 @@ class IColumn; M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 73fb28dc67a..5de4630758d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,8 +79,9 @@ static std::initializer_list LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr + auto read_buffer_creator = + [=] (bool /* restricted_seek */, const StoredObject & object) + -> std::unique_ptr { - LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); - - auto reader = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - - std::string answer(1000, ' '); - size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); - switch (read_settings.remote_fs_method) - { - case RemoteFSReadMethod::read: - { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - return std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */false); - } - case RemoteFSReadMethod::threadpool: - { - // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); - auto impl = std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - // auto impl3 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - - // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - // auto async_reader = std::make_unique( - // std::move(impl3), - // reader2, - // read_settings, - // global_context->getAsyncReadCounters(), - // global_context->getFilesystemReadPrefetchesLog()); - - // answer = std::string(1000, ' '); - // read_bytes = async_reader->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), - reader, - read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - } + return std::make_unique( + std::move(read_buffer_creator), + objects, + "file:", + modified_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */ false); } ReadSettings LocalObjectStorage::patchSettings(const ReadSettings & read_settings) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 740132f0981..364bd21a64e 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -66,7 +66,7 @@ StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings( const auto & settings = context->getSettingsRef(); return StorageObjectStorage::QuerySettings{ .truncate_on_insert = settings.engine_file_truncate_on_insert, - .create_new_file_on_insert = false, + .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8b800cbc343..3053a48b75b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,7 +461,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - read_settings.remote_fs_method = RemoteFSReadMethod::read; + // read_settings.remote_fs_method = RemoteFSReadMethod::read; LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index e49e14d2a0c..6491deef440 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,7 +40,10 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); + configuration.getNamespace(), + key, + configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName()); } void resolveSchemaAndFormat( From 436f6463c03b5cfc56af30e49faa6e192750a8f5 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:06:30 +0000 Subject: [PATCH 027/409] Add test --- contrib/grpc | 2 +- .../test_local_storage/configs/config.xml | 3 + tests/integration/test_local_storage/test.py | 243 +++++++++--------- tests/integration/tmp_data/example.csv | 5 + 4 files changed, 131 insertions(+), 122 deletions(-) create mode 100644 tests/integration/test_local_storage/configs/config.xml create mode 100644 tests/integration/tmp_data/example.csv diff --git a/contrib/grpc b/contrib/grpc index f5b7fdc2dff..1716359d2e2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df +Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml new file mode 100644 index 00000000000..bcbb107f0a2 --- /dev/null +++ b/tests/integration/test_local_storage/configs/config.xml @@ -0,0 +1,3 @@ + + /var/lib/clickhouse/ + \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 96d116ec6a2..d27fb746d12 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -6,148 +6,149 @@ import string import pytest from helpers.cluster import ClickHouseCluster -from azure.storage.blob import BlobServiceClient -from test_storage_azure_blob_storage.test import azure_query -NODE_NAME = "node" - - -def generate_cluster_def(port): - path = os.path.join( - os.path.dirname(os.path.realpath(__file__)), - "./_gen/disk_storage_conf.xml", - ) - os.makedirs(os.path.dirname(path), exist_ok=True) - with open(path, "w") as f: - f.write( - f""" - - - - object_storage - azure_blob_storage - plain_rewritable - http://azurite1:{port}/devstoreaccount1 - cont - true - devstoreaccount1 - Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== - 100000 - 100000 - 10 - 10 - - - - - -
- blob_storage_disk -
-
-
-
-
-
-""" - ) - return path - - -insert_values = [ - "(0,'data'),(1,'data')", - ",".join( - f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" - for i in range(10) - ), -] +from pathlib import Path @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): + global cluster try: cluster = ClickHouseCluster(__file__) - port = cluster.azurite_port - path = generate_cluster_def(port) cluster.add_instance( - NODE_NAME, - main_configs=[ - path, - ], - with_azurite=True, - stay_alive=True, + "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True ) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") + cluster.start() yield cluster + finally: cluster.shutdown() -def test_insert_select(cluster): - node = cluster.instances[NODE_NAME] +def process_result(line: str): + return sorted( + list( + map( + lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), + filter(lambda x: len(x) > 0, line.split("\n")), + ) + ) + ) - for index, value in enumerate(insert_values): - azure_query( - node, - """ - CREATE TABLE test_{} ( + +def test_local_engine(started_cluster): + node = started_cluster.instances["test_local_storage"] + node.query( + """ + CREATE TABLE test_0 ( id Int64, data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='blob_storage_policy' - """.format( - index - ), - ) + ) ENGINE=Local('/data/example.csv', 'CSV'); + """ + ) - azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """ + ) + result = node.query( + """ + select * from test_0; + """ + ) -def test_restart_server(cluster): - node = cluster.instances[NODE_NAME] + assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) + + error_got = node.query_and_get_error( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """ + ) + + print("Error got", error_got) + + node.query( + """ + SET engine_file_truncate_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """, + settings={"engine_file_truncate_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + node.query( + """ + SET local_create_new_file_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """, + settings={"local_create_new_file_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [ + (-1, "7"), + (0, "15"), + (1, "3"), + (4, "abc"), + (5, "arr"), + (9, "ty"), + ] == process_result(result) - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) node.restart_clickhouse() - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) - - -def test_drop_table(cluster): - node = cluster.instances[NODE_NAME] - - for index, value in enumerate(insert_values): - node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) - - port = cluster.env_variables["AZURITE_PORT"] - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + result = node.query( + """ + SELECT * FROM test_0; + """ ) - blob_service_client = BlobServiceClient.from_connection_string(connection_string) - containers = blob_service_client.list_containers() - for container in containers: - container_client = blob_service_client.get_container_client(container) - assert len(list(container_client.list_blobs())) == 0 + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + +def test_table_function(started_cluster): + with open("/tmp/example.csv", "w") as f: + f.write( + """id,data +1,Str1 +2,Str2""" + ) + node = started_cluster.instances["test_local_storage"] + + node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + + result = node.query( + """ + SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); + """ + ) + + print("Res5", result) + + assert [(1, "Str1"), (2, "Str2")] == process_result(result) + + # assert False diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv new file mode 100644 index 00000000000..93d6fb20f38 --- /dev/null +++ b/tests/integration/tmp_data/example.csv @@ -0,0 +1,5 @@ + +id,data +1,'Str1' +2,'Str2' + \ No newline at end of file From 27ab6aa8b52118b3373aaa9ff3ccb8320a5f5344 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:16:24 +0000 Subject: [PATCH 028/409] Remove logs --- .../Local/LocalObjectStorage.cpp | 14 +-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 53 +-------- .../Executors/PullingPipelineExecutor.cpp | 22 +--- .../StorageObjectStorageSource.cpp | 107 ++++-------------- 4 files changed, 35 insertions(+), 161 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 266e39f8f97..01ff2e2baf1 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,18 +1,15 @@ #include -#include -#include -#include -#include +#include +#include +#include #include #include +#include #include #include -#include -#include -#include #include -#include +#include namespace fs = std::filesystem; @@ -58,7 +55,6 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - return std::make_unique( std::move(read_buffer_creator), objects, diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 351f167b6b9..a6672e14e10 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,42 +200,10 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; - // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( - // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr - // { - // return std::make_unique( - // client.get(), - // uri.bucket, - // object_.remote_path, - // uri.version_id, - // settings_ptr->request_settings, - // disk_read_settings, - // /* use_external_buffer */ true, - // /* offset */ 0, - // /* read_until_position */ 0, - // restricted_seek); - // }; - - switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ false); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - return std::make_unique( std::move(read_buffer_creator), objects, @@ -244,30 +212,15 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - + case RemoteFSReadMethod::threadpool: + { auto impl = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", disk_read_settings, global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + /* use_external_buffer */true); auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index a4db27d3f37..25c15d40c9a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,32 +42,19 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { - LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); - if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } - LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); - if (!executor->checkTimeLimitSoft()) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); - - if (!executor->executeStep(&has_data_flag)) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); - - chunk = pulling_format->getChunk(); - - LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); - return true; } @@ -86,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 3053a48b75b..086482e330a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,18 +1,17 @@ #include "StorageObjectStorageSource.h" +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include "Common/logger_useful.h" +#include #include #include @@ -71,7 +70,6 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { - LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -134,7 +132,6 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_dag) { auto keys = configuration->getPaths(); @@ -146,19 +143,6 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } - - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); - for (auto && key : copy_configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); - } - - iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -187,11 +171,8 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); - while (true) { - LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); - if (isCancelled() || !reader) { if (reader) @@ -199,15 +180,10 @@ Chunk StorageObjectStorageSource::generate() break; } - LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); - Chunk chunk; if (reader->pull(chunk)) { - LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); - UInt64 num_rows = chunk.getNumRows(); - LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -255,9 +231,6 @@ Chunk StorageObjectStorageSource::generate() return chunk; } - LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -328,8 +301,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); - LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); - QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -354,17 +325,11 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; - LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); - - std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; - LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); - - if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -377,8 +342,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { - LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); - CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -389,12 +352,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); - LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); - auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); - std::string answer(1000, ' '); - size_t read_bytes = new_read_buf->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -460,17 +418,11 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( const auto & object_size = object_info.metadata->size_bytes; auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - - // read_settings.remote_fs_method = RemoteFSReadMethod::read; - - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); - read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; const bool object_too_small = object_size <= 2 * context_->getSettingsRef().max_download_buffer_size; - const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; /// User's object may change, don't cache it. @@ -479,28 +431,24 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - // if (use_prefetch) - // { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + if (use_prefetch) + { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( + StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - auto async_reader - = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; - // } - // else - // { - // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - - // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - // } + return async_reader; + } + else + { + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) @@ -678,18 +626,11 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { - LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); - for (auto && key : keys) - { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); - } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? - for (auto && key : keys) { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } From 45e0f0350e475503cdd2ac0a30d14ca2466fceb9 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:25:37 +0000 Subject: [PATCH 029/409] Fix some stuff --- .../integration/test_local_storage/files/example2.csv | 3 +++ tests/integration/test_local_storage/test.py | 10 +--------- tests/integration/tmp_data/example.csv | 5 ----- 3 files changed, 4 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/tmp_data/example.csv diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv new file mode 100644 index 00000000000..1abfb225937 --- /dev/null +++ b/tests/integration/test_local_storage/files/example2.csv @@ -0,0 +1,3 @@ +id,data +1,Str1 +2,Str2 \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index d27fb746d12..54c8016c376 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -68,8 +68,6 @@ def test_local_engine(started_cluster): """ ) - print("Error got", error_got) - node.query( """ SET engine_file_truncate_on_insert = 1; @@ -131,15 +129,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): - with open("/tmp/example.csv", "w") as f: - f.write( - """id,data -1,Str1 -2,Str2""" - ) node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") result = node.query( """ diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv deleted file mode 100644 index 93d6fb20f38..00000000000 --- a/tests/integration/tmp_data/example.csv +++ /dev/null @@ -1,5 +0,0 @@ - -id,data -1,'Str1' -2,'Str2' - \ No newline at end of file From 07d03c0c67437eaf70fe70199e71430d643302b6 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:30:46 +0000 Subject: [PATCH 030/409] Fix minor things --- src/IO/ReadHelpers.cpp | 10 +++------- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 6 ------ .../integration/test_local_storage/configs/config.xml | 2 +- .../integration/test_local_storage/files/example2.csv | 2 +- 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ddc2e912fb3..c771fced73a 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,7 +80,6 @@ UUID parseUUID(std::span src) return uuid; } - void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -89,15 +88,12 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); - throw Exception( - ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, - "Cannot parse input: expected {} {}", - out.str(), - buf.buffer().end() - buf.position()); + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); } + bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 0be929bc2c1..10fffd148be 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -125,12 +125,6 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { - LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } } std::string getName() const override { return name; } diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml index bcbb107f0a2..b4179ee51df 100644 --- a/tests/integration/test_local_storage/configs/config.xml +++ b/tests/integration/test_local_storage/configs/config.xml @@ -1,3 +1,3 @@ /var/lib/clickhouse/ - \ No newline at end of file + diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv index 1abfb225937..7b6e6d6bab1 100644 --- a/tests/integration/test_local_storage/files/example2.csv +++ b/tests/integration/test_local_storage/files/example2.csv @@ -1,3 +1,3 @@ id,data 1,Str1 -2,Str2 \ No newline at end of file +2,Str2 From f0ef101bc558400c1f38666a390862da71e1c0f2 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:40:30 +0000 Subject: [PATCH 031/409] Minor changes --- src/Storages/ObjectStorage/Local/Configuration.cpp | 1 - tests/integration/test_local_storage/test.py | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 364bd21a64e..d64b22769ad 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int LOGICAL_ERROR; } void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 54c8016c376..89c3c17e1f0 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -131,7 +131,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") + node.copy_file_to_container( + "test_local_storage/files/example2.csv", "/data/example2.csv" + ) result = node.query( """ From 57181a5a4871b251c3d1aa04a39d467a715645a4 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 14:47:57 +0000 Subject: [PATCH 032/409] Fix compilation bug --- src/Storages/ObjectStorage/registerStorageObjectStorage.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 13f2df7a0e7..4bc8cfa6a2f 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -10,8 +10,6 @@ namespace DB { -#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -71,8 +69,6 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } -#endif - #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { From c59949d057bba311f040f7b6386b1f6b481a23dd Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 22 Jul 2024 09:50:47 +0000 Subject: [PATCH 033/409] Add different iceberg tables --- .../DataLakes/registerDataLakeStorages.cpp | 60 +++++++++++++++++-- src/TableFunctions/ITableFunctionDataLake.h | 27 ++++++++- .../registerDataLakeTableFunctions.cpp | 27 +++++---- 3 files changed, 96 insertions(+), 18 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index 0fa6402e892..e75ab3201b8 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -2,10 +2,12 @@ #if USE_AWS_S3 -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include namespace DB @@ -22,6 +24,54 @@ void registerStorageIceberg(StorageFactory & factory) auto configuration = std::make_shared(); StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergS3", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergAzure", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); + + factory.registerStorage( + "IcebergLocal", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); @@ -29,7 +79,7 @@ void registerStorageIceberg(StorageFactory & factory) { .supports_settings = false, .supports_schema_inference = true, - .source_access_type = AccessType::S3, + .source_access_type = AccessType::FILE, }); } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index fe6e5b3e593..db8287f97bf 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -76,6 +76,21 @@ struct TableFunctionIcebergName static constexpr auto name = "iceberg"; }; +struct TableFunctionIcebergS3Name +{ + static constexpr auto name = "icebergS3"; +}; + +struct TableFunctionIcebergAzureName +{ + static constexpr auto name = "icebergAzure"; +}; + +struct TableFunctionIcebergLocalName +{ + static constexpr auto name = "icebergLocal"; +}; + struct TableFunctionDeltaLakeName { static constexpr auto name = "deltaLake"; @@ -86,14 +101,20 @@ struct TableFunctionHudiName static constexpr auto name = "hudi"; }; -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 using TableFunctionIceberg = ITableFunctionDataLake; +using TableFunctionIcebergS3 = ITableFunctionDataLake; +# endif +# if USE_AZURE_BLOB_STORAGE +using TableFunctionIcebergAzure = ITableFunctionDataLake; +# endif +using TableFunctionIcebergLocal = ITableFunctionDataLake; #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET using TableFunctionDeltaLake = ITableFunctionDataLake; #endif using TableFunctionHudi = ITableFunctionDataLake; #endif - } diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 15a6668f434..40561a1e075 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -4,24 +4,31 @@ namespace DB { -#if USE_AWS_S3 #if USE_AVRO void registerTableFunctionIceberg(TableFunctionFactory & factory) { +# if USE_AWS_S3 factory.registerFunction( - { - .documentation = - { - .description=R"(The table function can be used to read the Iceberg table stored on object store.)", + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"} - }, - .allow_readonly = false - }); + .categories{"DataLake"}}, + .allow_readonly = false}); + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); + +# endif +# if USE_AZURE_BLOB_STORAGE +# endif } #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET void registerTableFunctionDeltaLake(TableFunctionFactory & factory) { factory.registerFunction( From 4978869d2f709a9ad93cefc04cda43bcc739fb22 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 18:04:38 +0800 Subject: [PATCH 034/409] stash --- src/Functions/FunctionOverlay.cpp | 481 ++++++++++++++++++++++++++++++ 1 file changed, 481 insertions(+) create mode 100644 src/Functions/FunctionOverlay.cpp diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp new file mode 100644 index 00000000000..6160335ad79 --- /dev/null +++ b/src/Functions/FunctionOverlay.cpp @@ -0,0 +1,481 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ZERO_ARRAY_OR_TUPLE_INDEX; +} + +namespace +{ + +/// If 'is_utf8' - measure offset and length in code points instead of bytes. +/// Syntax: overlay(input, replace, offset[, length]) +template +class FunctionOverlay : public IFunction +{ +public: + static constexpr auto name = is_utf8 ? "OverlayUTF8" : "Overlay"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const size_t number_of_arguments = arguments.size(); + if (number_of_arguments < 3 || number_of_arguments > 4) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: " + "passed {}, should be 3 or 4", + getName(), + number_of_arguments); + + /// first argument is string + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected String", + arguments[0]->getName(), + getName()); + + /// second argument is string + if (!isString(arguments[1])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, expected String", + arguments[1]->getName(), + getName()); + + if (!isNativeNumber(arguments[2])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {}, expected (U)Int8|16|32|64", + arguments[2]->getName(), + getName()); + + if (number_of_arguments == 4 && !isNativeNumber(arguments[3])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, expected (U)Int8|16|32|64", + arguments[3]->getName(), + getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const size_t number_of_arguments = arguments.size(); + + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_offset = arguments[1].column; + ColumnPtr column_length; + if (number_of_arguments == 3) + column_length = arguments[2].column; + + const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); + const ColumnConst * column_length_const = nullptr; + if (number_of_arguments == 3) + column_length_const = checkAndGetColumn(column_length.get()); + + Int64 offset = 0; + Int64 length = 0; + if (column_offset_const) + offset = column_offset_const->getInt(0); + if (column_length_const) + length = column_length_const->getInt(0); + + auto res_col = ColumnString::create(); + auto & res_data = res_col->getChars(); + auto & res_offsets = res_col->getOffsets(); + } + +private: +template + void constantConstant( + size_t rows, + const StringRef & input, + const StringRef & replace, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + constantConstant(input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + size_t replace_size = replace.size; + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + else if constexpr (three_args) + { + valid_length = replace_size; + } + + size_t res_offset = 0; + size_t input_size = input.size; + for (size_t i = 0; i < rows; ++i) + { + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (!three_args && !length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void vectorConstant( + const ColumnString::Chars & input_data, + const ColumnString::Offsets & input_offsets, + const StringRef & replace, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + vectorConstant(input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + size_t replace_size = replace.size; + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + else if constexpr (three_args) + { + valid_length = replace_size; + } + + size_t rows = input_offsets.size(); + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t input_offset = input_offsets[i - 1]; + size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (!three_args && !length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void constantVector( + const StringRef & input, + const ColumnString::Chars & replace_data, + const ColumnString::Offsets & replace_offsets, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + constantVector(input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + + size_t rows = replace_offsets.size(); + size_t input_size = input.size; + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t replace_offset = replace_offsets[i - 1]; + size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (three_args) + { + // length = replace_size; + valid_length = replace_size; + } + else if constexpr (!length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void vectorVector( + const ColumnString::Chars & input_data, + const ColumnString::Offsets & input_offsets, + const ColumnString::Chars & replace_data, + const ColumnString::Offsets & replace_offsets, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + vectorVector( + input_data, + input_offsets, + replace_data, + replace_offsets, + column_offset, + column_length, + const_offset, + -1, + res_data, + res_offsets); + return; + } + + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + + size_t rows = input_offsets.size(); + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t input_offset = input_offsets[i - 1]; + size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t replace_offset = replace_offsets[i - 1]; + size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (three_args) + { + // length = replace_size; + valid_length = replace_size; + } + else if constexpr (!length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } +}; + +} + +REGISTER_FUNCTION(Overlay) +{ + factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::CaseSensitive); +} + +} From 81688e0efdf75a6a3923d6b95f09579d37e93e2a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 20:29:35 +0800 Subject: [PATCH 035/409] almost finish --- src/Functions/FunctionOverlay.cpp | 154 ++++++++++++++++++++++++++---- 1 file changed, 134 insertions(+), 20 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 6160335ad79..65af4d811f5 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -15,10 +15,8 @@ namespace DB namespace ErrorCodes { -extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int ZERO_ARRAY_OR_TUPLE_INDEX; } namespace @@ -86,32 +84,145 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const size_t number_of_arguments = arguments.size(); + bool three_args = number_of_arguments == 3; - ColumnPtr column_string = arguments[0].column; - ColumnPtr column_offset = arguments[1].column; + ColumnPtr column_offset = arguments[2].column; ColumnPtr column_length; - if (number_of_arguments == 3) - column_length = arguments[2].column; + if (!three_args) + column_length = arguments[3].column; const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); const ColumnConst * column_length_const = nullptr; - if (number_of_arguments == 3) + if (!three_args) column_length_const = checkAndGetColumn(column_length.get()); - Int64 offset = 0; - Int64 length = 0; + bool offset_is_const = false; + bool length_is_const = false; + Int64 offset = -1; + Int64 length = -1; if (column_offset_const) + { offset = column_offset_const->getInt(0); + offset_is_const = true; + } + if (column_length_const) + { length = column_length_const->getInt(0); + length_is_const = true; + } + auto res_col = ColumnString::create(); auto & res_data = res_col->getChars(); auto & res_offsets = res_col->getOffsets(); + res_offsets.resize_exact(input_rows_count); + + ColumnPtr column_input = arguments[0].column; + ColumnPtr column_replace = arguments[1].column; + + const auto * column_input_const = checkAndGetColumn(column_input.get()); + const auto * column_input_string = checkAndGetColumn(column_input.get()); + if (column_input_const) + { + StringRef input = column_input_const->getDataAt(0); + res_data.reserve(input.size * input_rows_count); + } + else + { + res_data.reserve(column_input_string->getChars().size()); + } + + const auto * column_replace_const = checkAndGetColumn(column_replace.get()); + const auto * column_replace_string = checkAndGetColumn(column_replace.get()); + bool input_is_const = column_input_const != nullptr; + bool replace_is_const = column_replace_const != nullptr; + +#define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ + if (input_is_const && replace_is_const) \ + constantConstant( \ + input_rows_count, \ + column_input_const->getDataAt(0), \ + column_replace_const->getDataAt(0), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else if (input_is_const) \ + constantVector( \ + column_input_const->getDataAt(0), \ + column_replace_string->getChars(), \ + column_replace_string->getOffsets(), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else if (replace_is_const) \ + vectorConstant( \ + column_input_string->getChars(), \ + column_input_string->getOffsets(), \ + column_replace_const->getDataAt(0), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else \ + vectorVector( \ + column_input_string->getChars(), \ + column_input_string->getOffsets(), \ + column_replace_string->getChars(), \ + column_replace_string->getOffsets(), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); + + if (three_args) + { + if (offset_is_const) + { + OVERLAY_EXECUTE_CASE(true, true, false) + } + else + { + OVERLAY_EXECUTE_CASE(true, false, false) + } + } + else + { + if (offset_is_const && length_is_const) + { + OVERLAY_EXECUTE_CASE(false, true, true) + } + else if (offset_is_const && !length_is_const) + { + OVERLAY_EXECUTE_CASE(false, true, false) + } + else if (!offset_is_const && length_is_const) + { + OVERLAY_EXECUTE_CASE(false, false, true) + } + else + { + OVERLAY_EXECUTE_CASE(false, false, false) + } + } +#undef OVERLAY_EXECUTE_CASE + + return res_col; } + private: -template + template void constantConstant( size_t rows, const StringRef & input, @@ -121,11 +232,12 @@ template ( + rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -194,7 +306,7 @@ template + template void vectorConstant( const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, @@ -204,11 +316,12 @@ template ( + input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -281,7 +394,7 @@ template + template void constantVector( const StringRef & input, const ColumnString::Chars & replace_data, @@ -291,11 +404,12 @@ template ( + input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -379,11 +493,11 @@ template ( + vectorVector( input_data, input_offsets, replace_data, From 9785f85ca32e3af5760a9bc24e998e7d050fc073 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 21:08:02 +0800 Subject: [PATCH 036/409] fix style --- src/Functions/FunctionOverlay.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 65af4d811f5..7d0e2e86de2 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -1,14 +1,10 @@ #include -#include #include -#include -#include #include #include #include #include -#include - +#include namespace DB { From f4138ee6c67bbdb82269a9087b1b054f33cb35a8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 10:20:27 +0800 Subject: [PATCH 037/409] fix bugs about corner cases --- src/Functions/FunctionOverlay.cpp | 89 +++++++++++++++++-------------- 1 file changed, 48 insertions(+), 41 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 7d0e2e86de2..d3ee7e1df6d 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -218,6 +218,26 @@ public: private: + /// input offset is 1-based, maybe negative + /// output result is 0-based valid offset, within [0, input_size] + static size_t getValidOffset(Int64 offset, size_t input_size) + { + if (offset > 0) + { + if (static_cast(offset) > input_size + 1) [[unlikely]] + return input_size; + else + return offset - 1; + } + else + { + if (input_size < -static_cast(offset)) [[unlikely]] + return 0; + else + return input_size + offset; + } + } + template void constantConstant( size_t rows, @@ -237,13 +257,10 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative + size_t input_size = input.size; size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } + valid_offset = getValidOffset(const_offset, input_size); size_t replace_size = replace.size; Int64 length = 0; // maybe negative @@ -258,14 +275,14 @@ private: valid_length = replace_size; } + Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; - size_t input_size = input.size; for (size_t i = 0; i < rows; ++i) { if constexpr (!offset_is_const) { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (!three_args && !length_is_const) @@ -274,7 +291,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -321,14 +338,6 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative - size_t valid_offset = 0; // start from 0, not negative - if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } - size_t replace_size = replace.size; Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative @@ -343,16 +352,22 @@ private: } size_t rows = input_offsets.size(); + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; - if constexpr (!offset_is_const) + if constexpr (offset_is_const) + { + valid_offset = getValidOffset(const_offset, input_size); + } + else { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (!three_args && !length_is_const) @@ -361,7 +376,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -409,13 +424,10 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative + size_t input_size = input.size; size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } + valid_offset = getValidOffset(const_offset, input_size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative @@ -426,7 +438,7 @@ private: } size_t rows = replace_offsets.size(); - size_t input_size = input.size; + Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -436,12 +448,11 @@ private: if constexpr (!offset_is_const) { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (three_args) { - // length = replace_size; valid_length = replace_size; } else if constexpr (!length_is_const) @@ -450,7 +461,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -507,15 +518,6 @@ private: return; } - - Int64 offset = 0; // start from 1, maybe negative - size_t valid_offset = 0; // start from 0, not negative - if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } - Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) @@ -525,6 +527,8 @@ private: } size_t rows = input_offsets.size(); + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -533,15 +537,18 @@ private: size_t replace_offset = replace_offsets[i - 1]; size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; - if constexpr (!offset_is_const) + if constexpr (offset_is_const) + { + valid_offset = getValidOffset(const_offset, input_size); + } + else { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (three_args) { - // length = replace_size; valid_length = replace_size; } else if constexpr (!length_is_const) @@ -550,7 +557,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); From fd3f0cf92b7800b171c5723541a329748a0dad1b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 14:17:58 +0800 Subject: [PATCH 038/409] support overlayUTF8 --- src/Functions/FunctionOverlay.cpp | 281 +++++++++++++----- .../0_stateless/03205_overlay.reference | 168 +++++++++++ tests/queries/0_stateless/03205_overlay.sql | 60 ++++ .../0_stateless/03206_overlay_utf8.reference | 168 +++++++++++ .../0_stateless/03206_overlay_utf8.sql | 60 ++++ 5 files changed, 665 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/03205_overlay.reference create mode 100644 tests/queries/0_stateless/03205_overlay.sql create mode 100644 tests/queries/0_stateless/03206_overlay_utf8.reference create mode 100644 tests/queries/0_stateless/03206_overlay_utf8.sql diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index d3ee7e1df6d..61d2df88ab1 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -3,8 +3,10 @@ #include #include #include +#include #include #include +#include namespace DB { @@ -15,6 +17,8 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +using namespace GatherUtils; + namespace { @@ -24,7 +28,7 @@ template class FunctionOverlay : public IFunction { public: - static constexpr auto name = is_utf8 ? "OverlayUTF8" : "Overlay"; + static constexpr auto name = is_utf8 ? "overlayUTF8" : "overlay"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -238,6 +242,15 @@ private: } } + /// get character count of a slice [data, data+bytes) + static size_t getSliceSize(const UInt8 * data, size_t bytes) + { + if constexpr (is_utf8) + return UTF8::countCodePoints(data, bytes); + else + return bytes; + } + template void constantConstant( size_t rows, @@ -257,13 +270,12 @@ private: return; } - size_t input_size = input.size; + size_t input_size = getSliceSize(reinterpret_cast(input.data), input.size); size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) valid_offset = getValidOffset(const_offset, input_size); - size_t replace_size = replace.size; - Int64 length = 0; // maybe negative + size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) { @@ -276,6 +288,9 @@ private: } Int64 offset = 0; // start from 1, maybe negative + Int64 length = 0; // maybe negative + const UInt8 * input_begin = reinterpret_cast(input.data); + const UInt8 * input_end = reinterpret_cast(input.data + input.size); size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -293,28 +308,57 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; + + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + + size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace.size); + res_offset += replace.size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -338,7 +382,7 @@ private: return; } - size_t replace_size = replace.size; + size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) @@ -358,7 +402,8 @@ private: for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; - size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_bytes = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_size = getSliceSize(&input_data[input_offset], input_bytes); if constexpr (offset_is_const) { @@ -378,29 +423,59 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15( - &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * input_begin = &input_data[input_offset]; + const auto * input_end = &input_data[input_offset + input_bytes]; + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + + size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace.size); + res_offset += replace.size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -424,7 +499,7 @@ private: return; } - size_t input_size = input.size; + size_t input_size = getSliceSize(reinterpret_cast(input.data), input.size); size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) valid_offset = getValidOffset(const_offset, input_size); @@ -438,12 +513,15 @@ private: } size_t rows = replace_offsets.size(); + const auto * input_begin = reinterpret_cast(input.data); + const auto * input_end = reinterpret_cast(input.data + input.size); Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { size_t replace_offset = replace_offsets[i - 1]; - size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_bytes = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_size = getSliceSize(&replace_data[replace_offset], replace_bytes); if constexpr (!offset_is_const) { @@ -463,28 +541,55 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_bytes); + res_offset += replace_bytes; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -533,9 +638,12 @@ private: for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; - size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_bytes = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_size = getSliceSize(&input_data[input_offset], input_bytes); + size_t replace_offset = replace_offsets[i - 1]; - size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_bytes = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_size = getSliceSize(&replace_data[replace_offset], replace_bytes); if constexpr (offset_is_const) { @@ -559,29 +667,58 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15( - &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * input_begin = &input_data[input_offset]; + const auto * input_end = &input_data[input_offset + input_bytes]; + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_bytes); + res_offset += replace_bytes; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference new file mode 100644 index 00000000000..9e79db2e131 --- /dev/null +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -0,0 +1,168 @@ +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql new file mode 100644 index 00000000000..b131312c934 --- /dev/null +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -0,0 +1,60 @@ +SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlay('Spark SQL', '_', 6) from numbers(3); +SELECT overlay(materialize('Spark SQL'), '_', 6) from numbers(3); +SELECT overlay('Spark SQL', materialize('_'), 6) from numbers(3); +SELECT overlay('Spark SQL', '_', materialize(6)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('_'), 6) from numbers(3); +SELECT overlay(materialize('Spark SQL'), '_', materialize(6)) from numbers(3); +SELECT overlay('Spark SQL', materialize('_'), materialize(6)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)) from numbers(3); + +SELECT overlay('Spark SQL', 'CORE', 7) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'CORE', 7) from numbers(3); +SELECT overlay('Spark SQL', materialize('CORE'), 7) from numbers(3); +SELECT overlay('Spark SQL', 'CORE', materialize(7)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)) from numbers(3); +SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)) from numbers(3); + +SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlay('Spark SQL', 'tructured', 2, 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), 2, 4) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', materialize(2), 4) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); diff --git a/tests/queries/0_stateless/03206_overlay_utf8.reference b/tests/queries/0_stateless/03206_overlay_utf8.reference new file mode 100644 index 00000000000..19878c97184 --- /dev/null +++ b/tests/queries/0_stateless/03206_overlay_utf8.reference @@ -0,0 +1,168 @@ +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark_SQL和CH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Spark ANSI SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH +Structured SQL和CH diff --git a/tests/queries/0_stateless/03206_overlay_utf8.sql b/tests/queries/0_stateless/03206_overlay_utf8.sql new file mode 100644 index 00000000000..00b756c8b5b --- /dev/null +++ b/tests/queries/0_stateless/03206_overlay_utf8.sql @@ -0,0 +1,60 @@ +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlayUTF8('Spark SQL和CH', '_', 6) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), '_', 6) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('_'), 6) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', '_', materialize(6)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), 6) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), '_', materialize(6)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('_'), materialize(6)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), materialize(6)) from numbers(3); + +SELECT overlayUTF8('Spark SQL和CH', 'CORE', 7) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'CORE', 7) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('CORE'), 7) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'CORE', materialize(7)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), 7) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'CORE', materialize(7)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('CORE'), materialize(7)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), materialize(7)) from numbers(3); + +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlayUTF8('Spark SQL和CH', 'tructured', 2, 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, 4) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, 4) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), 4) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), 2, 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', materialize(2), 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), materialize(2), 4) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, materialize(4)) from numbers(3); +SELECT overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), materialize(4)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); From c09c22b17575396e38fb45cb385dcc8a49f9a183 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 14:45:47 +0800 Subject: [PATCH 039/409] finish doc --- .../functions/string-replace-functions.md | 72 +++++++++++++++++++ ...new_functions_must_be_documented.reference | 2 + 2 files changed, 74 insertions(+) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 8793ebdd1a3..4e1f89fd974 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -223,3 +223,75 @@ SELECT translateUTF8('Münchener Straße', 'üß', 'us') AS res; │ Munchener Strase │ └──────────────────┘ ``` + +## overlay + +Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. + +**Syntax** + +```sql +overlay(s, replace, position[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `position`: An integer type [Int](../data-types/int.md). +- `length`: Optional. An integer type [Int](../data-types/int.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. + +**Example** + +```sql +SELECT overlay('Spark SQL', 'CORE', 7) AS res; +``` + +Result: + +```text + ┌─res────────┐ + │ Spark CORE │ + └────────────┘ +``` + +## overlayUTF8 + +Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +overlayUTF8(s, replace, position[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `position`: An integer type [Int](../data-types/int.md). +- `length`: Optional. An integer type [Int](../data-types/int.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. + +**Example** + +```sql +SELECT overlayUTF8('ClickHouse是一款OLAP数据库', '开源', 12, 2) AS res; +``` + +Result: + +```text +┌─res────────────────────────┐ +│ ClickHouse是开源OLAP数据库 │ +└────────────────────────────┘ +``` diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index a152066a460..ba9d3fb7a83 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -512,6 +512,8 @@ nullIf nullIn nullInIgnoreSet or +overlay +overlayUTF8 parseDateTime parseDateTime32BestEffort parseDateTime32BestEffortOrNull From c837541a7783f14780a7d2535dd6fa2cbf5effd5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 10:11:53 +0800 Subject: [PATCH 040/409] fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 943caf918d6..fa26cc0ff1f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2166,6 +2166,7 @@ outfile overcommit overcommitted overfitting +overlayUTF overparallelization packetpool packetsize From 3db505a1327fc5bf96c93f2a510436402be13f3b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 27 Jul 2024 12:53:38 +0200 Subject: [PATCH 041/409] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 1dda9e72084..dc3bf984cc6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -67,6 +67,7 @@ static std::initializer_list Date: Mon, 29 Jul 2024 13:03:21 +0200 Subject: [PATCH 042/409] Move setting to 24.8 version --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index dc3bf984cc6..41319ac7645 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,8 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.8", {{"input_format_try_infer_variants", false, false, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, + }}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, @@ -66,7 +68,6 @@ static std::initializer_list Date: Wed, 31 Jul 2024 10:29:12 +0000 Subject: [PATCH 043/409] Unifiing tests changes --- .../test/integration/runner/requirements.txt | 1 + .../ObjectStorage/Azure/Configuration.cpp | 10 +- .../DataLakes/registerDataLakeStorages.cpp | 2 +- .../helpers/{s3_tools.py => cloud_tools.py} | 78 +++-- .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 + .../configs/users.d/users.xml | 9 + .../test_iceberg_azure_storage/test.py | 291 ++++++++++++++++++ tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 214 ++++++++++--- tests/integration/test_storage_s3/test.py | 2 +- 12 files changed, 552 insertions(+), 68 deletions(-) rename tests/integration/helpers/{s3_tools.py => cloud_tools.py} (53%) create mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py create mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml create mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml create mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8a77d8abf77..db10398e73d 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,6 +64,7 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 +pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index f0a0a562b92..9730391d429 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -148,10 +148,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, { if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage AzureBlobStorage requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " - "[account_name, account_key, format, compression, structure)])"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage AzureBlobStorage requires 3 to {} arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure)])", + (with_structure ? 8 : 7)); } for (auto & engine_arg : engine_args) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index e75ab3201b8..f0bd51de375 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -54,7 +54,7 @@ void registerStorageIceberg(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true); return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/cloud_tools.py similarity index 53% rename from tests/integration/helpers/s3_tools.py rename to tests/integration/helpers/cloud_tools.py index 0c3538c3c39..534791b8bc5 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -2,30 +2,66 @@ from minio import Minio import glob import os import json +import shutil -def upload_directory(minio_client, bucket_name, local_path, s3_path): - result_files = [] - for local_file in glob.glob(local_path + "/**"): - if os.path.isfile(local_file): +from enum import Enum + + +class CloudUploader: + def upload_directory(self, local_path, remote_blob_path): + result_files = [] + # print(f"Arguments: {local_path}, {s3_path}") + # for local_file in glob.glob(local_path + "/**"): + # print("Local file: {}", local_file) + for local_file in glob.glob(local_path + "/**"): result_local_path = os.path.join(local_path, local_file) - result_s3_path = os.path.join(s3_path, local_file) - print(f"Putting file {result_local_path} to {result_s3_path}") - minio_client.fput_object( - bucket_name=bucket_name, - object_name=result_s3_path, - file_path=result_local_path, - ) - result_files.append(result_s3_path) - else: - files = upload_directory( - minio_client, - bucket_name, - os.path.join(local_path, local_file), - os.path.join(s3_path, local_file), - ) - result_files.extend(files) - return result_files + result_remote_blob_path = os.path.join(remote_blob_path, local_file) + if os.path.isfile(local_file): + self.upload_file(result_local_path, result_remote_blob_path) + result_files.append(result_remote_blob_path) + else: + files = self.upload_directory( + result_local_path, + result_remote_blob_path, + ) + result_files.extend(files) + return result_files + + +class S3Uploader(CloudUploader): + def __init__(self, minio_client, bucket_name): + self.minio_client = minio_client + self.bucket_name = bucket_name + + def upload_file(self, local_path, remote_blob_path): + self.minio_client.fput_object( + bucket_name=self.bucket_name, + object_name=remote_blob_path, + file_path=local_path, + ) + + +class LocalUploader(CloudUploader): + def __init__(self): + pass + + def upload_file(self, local_path, remote_blob_path): + if local_path != remote_blob_path: + shutil.copyfile(local_path, remote_blob_path) + + +class AzureUploader(CloudUploader): + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_name = container_name + + def upload_file(self, local_path, remote_blob_path): + blob_client = self.blob_service_client.get_blob_client( + container=self.container_name, blob=remote_blob_path + ) + with open(local_path, "rb") as data: + blob_client.upload_blob(data, overwrite=True) def get_file_contents(minio_client, bucket, s3_path): diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml new file mode 100644 index 00000000000..d4c54e2d13d --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/ + minio + minio123 + + + diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py new file mode 100644 index 00000000000..f96f8acfaaf --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -0,0 +1,291 @@ +import helpers.client +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.test_tools import TSV + +import pyspark +import logging +import os +import json +import pytest +import time +import glob +import uuid +import os + +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + +from pyspark.sql.types import ( + StructType, + StructField, + StringType, + IntegerType, + DateType, + TimestampType, + BooleanType, + ArrayType, +) +from pyspark.sql.functions import current_timestamp +from datetime import datetime +from pyspark.sql.functions import monotonically_increasing_id, row_number +from pyspark.sql.window import Window +from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 +from minio.deleteobjects import DeleteObject + +from tests.integration.helpers.cloud_tools import ( + prepare_s3_bucket, + upload_directory, + get_file_contents, + list_s3_objects, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def get_spark(): + builder = ( + pyspark.sql.SparkSession.builder.appName("spark_test") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.iceberg.spark.SparkSessionCatalog", + ) + .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") + .config( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", + ) + .master("local") + ) + return builder.master("local").getOrCreate() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=["configs/config.d/named_collections.xml"], + user_configs=["configs/users.d/users.xml"], + with_minio=True, + stay_alive=True, + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + cluster.spark_session = get_spark() + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def write_iceberg_from_file( + spark, path, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( + partition_by + ).tableProperty("format-version", format_version).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).append() + + +def write_iceberg_from_df( + spark, df, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).partitionedBy(partition_by).using("iceberg").create() + else: + df.writeTo(table_name).append() + + +def generate_data(spark, start, end): + a = spark.range(start, end, 1).toDF("a") + b = spark.range(start + 1, end + 1, 1).toDF("b") + b = b.withColumn("b", b["b"].cast(StringType())) + + a = a.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + b = b.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + + df = a.join(b, on=["row_index"]).drop("row_index") + return df + + +def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + + +def create_initial_data_file( + cluster, node, query, table_name, compression_method="none" +): + node.query( + f""" + INSERT INTO TABLE FUNCTION + file('{table_name}.parquet') + SETTINGS + output_format_parquet_compression_method='{compression_method}', + s3_truncate_on_insert=1 {query} + FORMAT Parquet""" + ) + user_files_path = os.path.join( + SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" + ) + result_path = f"{user_files_path}/{table_name}.parquet" + return result_path + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_single_iceberg_file(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_single_iceberg_file_" + format_version + + inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" + parquet_data_path = create_initial_data_file( + started_cluster, instance, inserted_data, TABLE_NAME + ) + + write_iceberg_from_file( + spark, parquet_data_path, TABLE_NAME, format_version=format_version + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + for bucket in minio_client.list_buckets(): + for object in minio_client.list_objects(bucket.name, recursive=True): + print("Object: ", object.object_name) + extension = object.object_name.split(".")[-1] + print("File extension: ", extension) + try: + response = minio_client.get_object( + object.bucket_name, object.object_name + ) + + if extension == "avro": + avro_bytes = response.read() + + # Use BytesIO to create a file-like object from the byte string + avro_file = io.BytesIO(avro_bytes) + + # Read the Avro data + reader = avro.datafile.DataFileReader( + avro_file, avro.io.DatumReader() + ) + records = [record for record in reader] + + # Close the reader + reader.close() + + # Now you can work with the records + for record in records: + # print(json.dumps(record, indent=4, sort_keys=True)) + print(str(record)) + # my_json = ( + # str(record) + # .replace("'", '"') + # .replace("None", "null") + # .replace('b"', '"') + # ) + # print(my_json) + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + elif extension == "json": + my_bytes_value = response.read() + my_json = my_bytes_value.decode("utf8").replace("'", '"') + data = json.loads(my_json) + s = json.dumps(data, indent=4, sort_keys=True) + print(s) + elif extension == "parquet": + # print("To be continued...") + # # Your byte string containing the Parquet data + # parquet_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # parquet_file = io.BytesIO(parquet_bytes) + + # # Read the Parquet data into a PyArrow Table + # table = pq.read_table(parquet_file) + + # # Convert the PyArrow Table to a Pandas DataFrame + # df = table.to_pandas() + + # # Now you can work with s DataFrame + # print(df) + parquet_bytes = ( + response.read() + ) # Replace with your actual byte string + + # Create a temporary file and write the byte string to it + with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + tmp_file.write(parquet_bytes) + tmp_file_path = tmp_file.name + + # Read the Parquet file using PySpark + df = spark.read.parquet(tmp_file_path) + + # Show the DataFrame + print(df.toPandas()) + else: + print(response.read()) + + finally: + print("----------------") + response.close() + response.release_conn() + + create_iceberg_table(instance, TABLE_NAME) + + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( + inserted_data + ) + + assert 0 == 1 diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d3dd7cfe52a..aaff1414d8d 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -28,7 +28,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 0c3fbfb3cda..750b77b29f3 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7762d17b96f..7f83846bd89 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,6 +12,14 @@ import glob import uuid import os +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + from pyspark.sql.types import ( StructType, StructField, @@ -29,11 +37,13 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, - upload_directory, get_file_contents, list_s3_objects, + S3Uploader, + AzureUploader, + LocalUploader, ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -67,6 +77,7 @@ def started_cluster(): main_configs=["configs/config.d/named_collections.xml"], user_configs=["configs/users.d/users.xml"], with_minio=True, + with_azurite=True, stay_alive=True, ) @@ -77,6 +88,15 @@ def started_cluster(): logging.info("S3 bucket created") cluster.spark_session = get_spark() + cluster.default_s3_uploader = S3Uploader( + cluster.minio_client, cluster.minio_bucket + ) + + container_name = "my_container" + + cluster.default_azurite_uploader = AzureUploader( + cluster.blob_service_client, container_name + ) yield cluster @@ -142,13 +162,25 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) +def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): + if storage_type == "local": + pass + elif storage_type == "s3": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + elif storage_type == "azure": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) def create_initial_data_file( @@ -170,32 +202,134 @@ def create_initial_data_file( return result_path +def default_upload_directory(started_cluster, storage_type, local_path, remote_path): + if storage_type == "local": + return LocalUploader().upload_directory(local_path, remote_path) + elif storage_type == "s3": + return started_cluster.default_s3_uploader.upload_directory( + local_path, remote_path + ) + elif storage_type == "azure": + return started_cluster.default_azure_uploader.upload_directory( + local_path, remote_path + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) + + @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3"]) +def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket TABLE_NAME = "test_single_iceberg_file_" + format_version - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME + write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) + + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) + # for bucket in minio_client.list_buckets(): + # for object in minio_client.list_objects(bucket.name, recursive=True): + # print("Object: ", object.object_name) + # extension = object.object_name.split(".")[-1] + # print("File extension: ", extension) + # try: + # response = minio_client.get_object( + # object.bucket_name, object.object_name + # ) - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) + # if extension == "avro": + # avro_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # avro_file = io.BytesIO(avro_bytes) + + # # Read the Avro data + # reader = avro.datafile.DataFileReader( + # avro_file, avro.io.DatumReader() + # ) + # records = [record for record in reader] + + # # Close the reader + # reader.close() + + # # Now you can work with the records + # for record in records: + # # print(json.dumps(record, indent=4, sort_keys=True)) + # print(str(record)) + # # my_json = ( + # # str(record) + # # .replace("'", '"') + # # .replace("None", "null") + # # .replace('b"', '"') + # # ) + # # print(my_json) + # # data = json.loads(my_json) + # # s = json.dumps(data, indent=4, sort_keys=True) + # # print(s) + # elif extension == "json": + # my_bytes_value = response.read() + # my_json = my_bytes_value.decode("utf8").replace("'", '"') + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + # elif extension == "parquet": + # # print("To be continued...") + # # # Your byte string containing the Parquet data + # # parquet_bytes = response.read() + + # # # Use BytesIO to create a file-like object from the byte string + # # parquet_file = io.BytesIO(parquet_bytes) + + # # # Read the Parquet data into a PyArrow Table + # # table = pq.read_table(parquet_file) + + # # # Convert the PyArrow Table to a Pandas DataFrame + # # df = table.to_pandas() + + # # # Now you can work with s DataFrame + # # print(df) + # parquet_bytes = ( + # response.read() + # ) # Replace with your actual byte string + + # # Create a temporary file and write the byte string to it + # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + # tmp_file.write(parquet_bytes) + # tmp_file_path = tmp_file.name + + # # Read the Parquet file using PySpark + # df = spark.read.parquet(tmp_file_path) + + # # Show the DataFrame + # print(df.toPandas()) + # else: + # print(response.read()) + + # finally: + # print("----------------") + # response.close() + # response.release_conn() create_iceberg_table(instance, TABLE_NAME) + + # print("Debug Print") + + # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) + + # print(instance.query("SELECT number FROM numbers(100)")) + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data + "SELECT number, toString(number + 1) FROM numbers(100)" ) + # assert 0 == 1 + @pytest.mark.parametrize("format_version", ["1", "2"]) def test_partition_by(started_cluster, format_version): @@ -215,7 +349,7 @@ def test_partition_by(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -240,7 +374,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', @@ -260,7 +394,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) assert len(files) == 9 @@ -302,7 +436,9 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory(minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "") + upload_directory( + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + ) create_iceberg_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 @@ -345,7 +481,7 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -354,7 +490,7 @@ def test_delete_files(started_cluster, format_version): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -369,14 +505,14 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @@ -399,7 +535,7 @@ def test_evolved_schema(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -410,7 +546,7 @@ def test_evolved_schema(started_cluster, format_version): spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -437,7 +573,7 @@ def test_row_based_deletes(started_cluster): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -446,7 +582,7 @@ def test_row_based_deletes(started_cluster): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -472,7 +608,7 @@ def test_schema_inference(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, format) @@ -527,7 +663,7 @@ def test_metadata_file_selection(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -559,7 +695,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -586,7 +722,7 @@ def test_restart_broken(started_cluster): write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, bucket=bucket) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 @@ -614,7 +750,7 @@ def test_restart_broken(started_cluster): minio_client.make_bucket(bucket) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 40cbf4b44a6..fea828dc3a5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.s3_tools import prepare_s3_bucket +from helpers.cloud_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 76847d9b4c27ec3011ef5866c3ea7b30fdcd2f08 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 2 Aug 2024 22:43:05 +0000 Subject: [PATCH 044/409] Fix(asan) : access destroyed shared context from handleCrash() --- programs/server/Server.cpp | 1 + src/Daemon/BaseDaemon.cpp | 2 -- src/Interpreters/Context.cpp | 27 +++++++++++++++++---------- src/Interpreters/Context.h | 8 ++++++-- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 20db4c2773c..5b9bb8b989d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -977,6 +977,7 @@ try /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. */ + global_context->resetSharedContext(); global_context.reset(); shared_context.reset(); LOG_DEBUG(log, "Destroyed global context."); diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index e7ae8ea5a1d..f74bd5e122c 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -23,9 +23,7 @@ #include #include -#include #include -#include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..743e762c81a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -892,6 +892,12 @@ ContextData::ContextData(const ContextData &o) : { } +void ContextData::resetSharedContext() +{ + std::lock_guard lock(mutex_shared_context); + shared = nullptr; +} + Context::Context() = default; Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} @@ -913,14 +919,6 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) return res; } -void Context::initGlobal() -{ - assert(!global_context_instance); - global_context_instance = shared_from_this(); - DatabaseCatalog::init(shared_from_this()); - EventNotifier::init(); -} - SharedContextHolder Context::createShared() { return SharedContextHolder(std::make_unique()); @@ -2691,7 +2689,11 @@ void Context::makeSessionContext() void Context::makeGlobalContext() { - initGlobal(); + assert(!global_context_instance); + global_context_instance = shared_from_this(); + DatabaseCatalog::init(shared_from_this()); + EventNotifier::init(); + global_context = shared_from_this(); } @@ -4084,8 +4086,13 @@ void Context::initializeTraceCollector() } /// Call after unexpected crash happen. -void Context::handleCrash() const TSA_NO_THREAD_SAFETY_ANALYSIS +void Context::handleCrash() const { + std::lock_guard lock(mutex_shared_context); + if (!shared) + return; + + SharedLockGuard lock2(shared->mutex); if (shared->system_logs) shared->system_logs->handleCrash(); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..9ab7e9169c4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,6 +490,8 @@ public: KitchenSink kitchen_sink; + void resetSharedContext(); + protected: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -527,6 +529,10 @@ protected: mutable ThrottlerPtr local_write_query_throttler; /// A query-wide throttler for local IO writes mutable ThrottlerPtr backups_query_throttler; /// A query-wide throttler for BACKUPs + + mutable std::mutex mutex_shared_context; /// mutex to avoid accessing destroyed shared context pointer + /// some Context methods can be called after the shared context is destroyed + /// example, Context::handleCrash() method - called from signal handler }; /** A set of known objects that can be used in the query. @@ -1385,8 +1391,6 @@ private: ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard & lock); - void initGlobal(); - void setUserID(const UUID & user_id_); void setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user); From e2b74e4176dca494c07c69b4dc3f4435c29aad3e Mon Sep 17 00:00:00 2001 From: sunny19930321 Date: Mon, 5 Aug 2024 19:32:45 +0800 Subject: [PATCH 045/409] feat: add delete from in pastition cmd --- docs/en/sql-reference/statements/delete.md | 2 +- src/Interpreters/InterpreterDeleteQuery.cpp | 8 +++++++ src/Parsers/ASTDeleteQuery.cpp | 6 +++++ src/Parsers/ASTDeleteQuery.h | 5 ++++ src/Parsers/ParserDeleteQuery.cpp | 12 ++++++++++ ..._lightweight_delete_in_partition.reference | 4 ++++ .../02352_lightweight_delete_in_partition.sql | 23 +++++++++++++++++++ 7 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index a52b7204c30..fe3868bcab4 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -10,7 +10,7 @@ title: The Lightweight DELETE Statement The lightweight `DELETE` statement removes rows from the table `[db.]table` that match the expression `expr`. It is only available for the *MergeTree table engine family. ``` sql -DELETE FROM [db.]table [ON CLUSTER cluster] WHERE expr; +DELETE FROM [db.]table [ON CLUSTER cluster] [IN PARTITION partition_expr] WHERE expr; ``` It is called "lightweight `DELETE`" to contrast it to the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command, which is a heavyweight process. diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 291c8e19db0..69aae3ae909 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -79,6 +79,14 @@ BlockIO InterpreterDeleteQuery::execute() "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + " UPDATE `_row_exists` = 0 WHERE " + serializeAST(*delete_query.predicate); + if (delete_query.partition) + { + alter_query = + "ALTER TABLE " + table->getStorageID().getFullTableName() + + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + + " UPDATE `_row_exists` = 0 IN PARTITION " + serializeAST(*delete_query.partition) + " WHERE " + + serializeAST(*delete_query.predicate); + } ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( diff --git a/src/Parsers/ASTDeleteQuery.cpp b/src/Parsers/ASTDeleteQuery.cpp index 67f3a85c9a5..434cc344a5a 100644 --- a/src/Parsers/ASTDeleteQuery.cpp +++ b/src/Parsers/ASTDeleteQuery.cpp @@ -45,6 +45,12 @@ void ASTDeleteQuery::formatQueryImpl(const FormatSettings & settings, FormatStat formatOnCluster(settings); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h index cc115a366db..6adc9b6a1c8 100644 --- a/src/Parsers/ASTDeleteQuery.h +++ b/src/Parsers/ASTDeleteQuery.h @@ -19,6 +19,11 @@ public: return removeOnCluster(clone(), params.default_database); } + /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE, DELETE FROM queries. + * The value or ID of the partition is stored here. + */ + ASTPtr partition; + ASTPtr predicate; protected: diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 00fac45c8ed..65d593b6c1b 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -15,11 +16,14 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_delete(Keyword::DELETE); ParserKeyword s_from(Keyword::FROM); + ParserKeyword s_in_partition(Keyword::IN_PARTITION); ParserKeyword s_where(Keyword::WHERE); ParserExpression parser_exp_elem; ParserKeyword s_settings(Keyword::SETTINGS); ParserKeyword s_on{Keyword::ON}; + ParserPartition parser_partition; + if (s_delete.ignore(pos, expected)) { if (!s_from.ignore(pos, expected)) @@ -36,6 +40,12 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->cluster = cluster_str; } + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, query->partition, expected)) + return false; + } + if (!s_where.ignore(pos, expected)) return false; @@ -52,6 +62,8 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; + if (query->partition) + query->children.push_back(query->partition); if (query->predicate) query->children.push_back(query->predicate); diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference new file mode 100644 index 00000000000..2bd7b51cea0 --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference @@ -0,0 +1,4 @@ +200 +200 +100 +100 diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql new file mode 100644 index 00000000000..08cf5816f74 --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS t_merge_tree SYNC; +DROP TABLE IF EXISTS t_replicated_merge_tree SYNC; + +CREATE TABLE t_merge_tree(time Date, id String , name String) ENGINE = MergeTree() PARTITION BY time ORDER BY id; +CREATE TABLE t_replicated_merge_tree(time Date, id String, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','1') PARTITION BY time ORDER BY id; + +INSERT INTO t_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); +INSERT INTO t_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); + +INSERT INTO t_replicated_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); +INSERT INTO t_replicated_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); + +SELECT COUNT() FROM t_merge_tree; +SELECT COUNT() FROM t_replicated_merge_tree; + +DELETE FROM t_merge_tree IN PARTITION '2024-08-01' WHERE id = '1'; +DELETE FROM t_replicated_merge_tree IN PARTITION '2024-08-01' WHERE id = '1'; + +SELECT COUNT() FROM t_merge_tree; +SELECT COUNT() FROM t_replicated_merge_tree; + +DROP TABLE t_merge_tree SYNC; +DROP TABLE t_replicated_merge_tree SYNC; From 1b251fe08837aee56fe541bf3ef54ff647650869 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:10:31 +0000 Subject: [PATCH 046/409] Allow to specify min and max for random settings in the test --- docs/en/development/tests.md | 22 +++++ tests/clickhouse-test | 92 ++++++++++++++++--- ...mic_read_subcolumns_compact_merge_tree.sql | 1 + ...ynamic_read_subcolumns_wide_merge_tree.sql | 1 + ...merges_1_horizontal_compact_merge_tree.sql | 4 + ..._merges_1_horizontal_compact_wide_tree.sql | 2 + ...c_merges_1_vertical_compact_merge_tree.sql | 2 + ...amic_merges_1_vertical_wide_merge_tree.sql | 2 + ...merges_2_horizontal_compact_merge_tree.sql | 1 + ...ic_merges_2_horizontal_wide_merge_tree.sql | 1 + ...c_merges_2_vertical_compact_merge_tree.sql | 1 + ...amic_merges_2_vertical_wide_merge_tree.sql | 1 + ...sted_dynamic_merges_compact_horizontal.sql | 1 + ...nested_dynamic_merges_compact_vertical.sql | 1 + ..._nested_dynamic_merges_wide_horizontal.sql | 1 + ...38_nested_dynamic_merges_wide_vertical.sql | 1 + 16 files changed, 122 insertions(+), 12 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..f0afa983fec 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -91,6 +91,28 @@ SELECT 1 In addition to the above settings, you can use `USE_*` flags from `system.build_options` to define usage of particular ClickHouse features. For example, if your test uses a MySQL table, you should add a tag `use-mysql`. +### Specifying limits for random settings + +A test can specify minimum and maximum allowed values for settings that can be randomized during test run. + +For `.sh` tests limits are written as a comment on the line next to tags or on the second line if no tags are specified: + +```bash +#!/usr/bin/env bash +# Tags: no-fasttest +# Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +``` + +For `.sql` tests tags are placed as a SQL comment in the line next to tags or in the first line: + +```sql +-- Tags: no-fasttest +-- Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +SELECT 1 +``` + +If you need to specify only one limit, you can use `None` for another one. + ### Choosing the Test Name The name of the test starts with a five-digit prefix followed by a descriptive name, such as `00422_hash_function_constexpr.sql`. To choose the prefix, find the largest prefix already present in the directory, and increment it by one. In the meantime, some other tests might be added with the same numeric prefix, but this is OK and does not lead to any problems, you don't have to change it later. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a29c786e998..ea488e7c3dd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -39,6 +39,7 @@ from errno import ESRCH from subprocess import PIPE, Popen from time import sleep, time from typing import Dict, List, Optional, Set, Tuple, Union +from ast import literal_eval as make_tuple try: import termcolor # type: ignore @@ -1068,9 +1069,25 @@ class TestCase: return description + "\n" + def apply_random_settings_limits(self, random_settings): + print("Random settings limits:", self.random_settings_limits) + for setting in random_settings: + if setting in self.random_settings_limits: + min = self.random_settings_limits[setting][0] + if min and random_settings[setting] < min: + random_settings[setting] = min + max = self.random_settings_limits[setting][1] + if max and random_settings[setting] > max: + random_settings[setting] = max + def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name self.tags: Set[str] = suite.all_tags[case] if case in suite.all_tags else set() + self.random_settings_limits = ( + suite.all_random_settings_limits[case] + if case in suite.all_random_settings_limits + else dict() + ) for tag in os.getenv("GLOBAL_TAGS", "").split(","): self.tags.add(tag.strip()) @@ -1112,11 +1129,13 @@ class TestCase: if self.randomize_settings: self.random_settings = SettingsRandomizer.get_random_settings(args) + self.apply_random_settings_limits(self.random_settings) if self.randomize_merge_tree_settings: self.merge_tree_random_settings = ( MergeTreeSettingsRandomizer.get_random_settings(args) ) + self.apply_random_settings_limits(self.merge_tree_random_settings) self.base_url_params = ( os.environ["CLICKHOUSE_URL_PARAMS"] @@ -1900,7 +1919,9 @@ class TestSuite: return test_name @staticmethod - def read_test_tags(suite_dir: str, all_tests: List[str]) -> Dict[str, Set[str]]: + def read_test_tags_and_random_settings_limits( + suite_dir: str, all_tests: List[str] + ) -> (Dict[str, Set[str]], Dict[str, Dict[str, Tuple[int, int]]]): def get_comment_sign(filename): if filename.endswith(".sql") or filename.endswith(".sql.j2"): return "--" @@ -1925,22 +1946,48 @@ class TestSuite: tags = {tag.strip() for tag in tags} return tags + def parse_random_settings_limits_from_line( + line, comment_sign + ) -> Dict[str, Tuple[int, int]]: + if not line.startswith(comment_sign): + return {} + random_settings_limits_str = line[ + len(comment_sign) : + ].lstrip() # noqa: ignore E203 + random_settings_limits_prefix = "Random settings limits:" + if not random_settings_limits_str.startswith(random_settings_limits_prefix): + return {} + random_settings_limits_str = random_settings_limits_str[ + len(random_settings_limits_prefix) : + ] # noqa: ignore E203 + # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' + random_settings_limits = re.findall( + "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str + ) + random_settings_limits = { + pair[0]: make_tuple(pair[1]) for pair in random_settings_limits + } + return random_settings_limits + def is_shebang(line: str) -> bool: return line.startswith("#!") def find_tag_line(file): - for line in file: + line = file.readline() + while line != "": line = line.strip() if line and not is_shebang(line): return line + line = file.readline() return "" - def load_tags_from_file(filepath): + def load_tags_and_random_settings_limits_from_file(filepath): comment_sign = get_comment_sign(filepath) need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: tag_line = find_tag_line(file) + next_line = file.readline() except UnicodeDecodeError: return [] try: @@ -1950,21 +1997,35 @@ class TestSuite: need_query_params = True except UnicodeDecodeError: pass - parsed_tags = parse_tags_from_line(tag_line, comment_sign) - if need_query_params: - parsed_tags.add("need-query-parameters") - return parsed_tags + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + random_settings_limits_line = next_line if parsed_tags else tag_line + random_settings_limits = parse_random_settings_limits_from_line( + random_settings_limits_line, comment_sign + ) + return parsed_tags, random_settings_limits all_tags = {} + all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags = load_tags_from_file(os.path.join(suite_dir, test_name)) + ( + tags, + random_settings_limits, + ) = load_tags_and_random_settings_limits_from_file( + os.path.join(suite_dir, test_name) + ) if tags: all_tags[test_name] = tags + if random_settings_limits: + all_random_settings_limits[test_name] = random_settings_limits elapsed = (datetime.now() - start_time).total_seconds() if elapsed > 1: - print(f"Tags for suite {suite_dir} read in {elapsed:.2f} seconds") - return all_tags + print( + f"Tags and random settings limits for suite {suite_dir} read in {elapsed:.2f} seconds" + ) + return all_tags, all_random_settings_limits def __init__(self, args, suite_path: str, suite_tmp_path: str, suite: str): self.args = args @@ -1994,9 +2055,16 @@ class TestSuite: self.all_tests: List[str] = self.get_tests_list( self.tests_in_suite_key_func, filter_func ) - self.all_tags: Dict[str, Set[str]] = self.read_test_tags( - self.suite_path, self.all_tests + + all_tags_and_random_settings_limits = ( + self.read_test_tags_and_random_settings_limits( + self.suite_path, self.all_tests + ) ) + self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql index ddfba4418bd..822393d3c78 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 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..2394893dc8b 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 @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index d2c787040e5..7c2e7c3d2be 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; @@ -31,3 +33,5 @@ optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); drop table test; + +select 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index f99bf771608..aa62435188a 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index be81596d043..bfc7bb9d206 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index f6396af42a8..233667db0a7 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index e133ac3001f..48a6a55378c 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index d527081b763..44b298b1c35 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index ebccfb77922..f42150720b3 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index 104d6018e41..ee4ff6af162 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 1d5c63dcdf1..e0636f053df 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index 2bffe35c577..edfad295e9a 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index fb686091ebb..79d488ec253 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index ed195452d56..e2a453b867a 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From d3dc17453377368defd80cda9f4b95dda6adc9df Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:15:11 +0000 Subject: [PATCH 047/409] Remove log --- tests/clickhouse-test | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 389193836bf..5fcb9fb80f1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -682,7 +682,6 @@ class FailureReason(enum.Enum): BUILD = "not running for current build" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" SHARED_MERGE_TREE = "no-shared-merge-tree" - DISTRIBUTED_CACHE = "distributed-cache" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1071,7 +1070,6 @@ class TestCase: return description + "\n" def apply_random_settings_limits(self, random_settings): - print("Random settings limits:", self.random_settings_limits) for setting in random_settings: if setting in self.random_settings_limits: min = self.random_settings_limits[setting][0] @@ -1211,9 +1209,6 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB - elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: - return FailureReason.DISTRIBUTED_CACHE - elif ( tags and ("atomic-database" in tags) @@ -1251,11 +1246,6 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and ( - 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) - ): - return FailureReason.SKIP - elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -2295,6 +2285,7 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) + break except TimeoutError: break finally: @@ -3280,12 +3271,6 @@ def parse_args(): default=False, help="Run tests over s3 storage", ) - parser.add_argument( - "--distributed-cache", - action="store_true", - default=False, - help="Run tests with enabled distributed cache", - ) parser.add_argument( "--azure-blob-storage", action="store_true", From 18a7a82458ce7ec3f12f7b6751699f119769ed55 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:16:18 +0000 Subject: [PATCH 048/409] Better formatting --- tests/clickhouse-test | 33 +++++++++++---------------------- 1 file changed, 11 insertions(+), 22 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5fcb9fb80f1..bcb8a12625b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1950,15 +1950,11 @@ class TestSuite: ) -> Dict[str, Tuple[int, int]]: if not line.startswith(comment_sign): return {} - random_settings_limits_str = line[ - len(comment_sign) : - ].lstrip() # noqa: ignore E203 + random_settings_limits_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 random_settings_limits_prefix = "Random settings limits:" if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} - random_settings_limits_str = random_settings_limits_str[ - len(random_settings_limits_prefix) : - ] # noqa: ignore E203 + random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' random_settings_limits = re.findall( "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str @@ -1996,25 +1992,20 @@ class TestSuite: need_query_params = True except UnicodeDecodeError: pass - parsed_tags = parse_tags_from_line(tag_line, comment_sign) - if need_query_params: - parsed_tags.add("need-query-parameters") - random_settings_limits_line = next_line if parsed_tags else tag_line - random_settings_limits = parse_random_settings_limits_from_line( - random_settings_limits_line, comment_sign - ) + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + random_settings_limits_line = next_line if parsed_tags else tag_line + random_settings_limits = parse_random_settings_limits_from_line( + random_settings_limits_line, comment_sign + ) return parsed_tags, random_settings_limits all_tags = {} all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - ( - tags, - random_settings_limits, - ) = load_tags_and_random_settings_limits_from_file( - os.path.join(suite_dir, test_name) - ) + tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2061,9 +2052,7 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 self.sequential_tests = [] self.parallel_tests = [] From 74a2976810b86086819ee8e6ee1f110ab1e70a37 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:13:03 +0000 Subject: [PATCH 049/409] Fix pylint --- tests/clickhouse-test | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index bcb8a12625b..84f33860484 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1072,12 +1072,12 @@ class TestCase: def apply_random_settings_limits(self, random_settings): for setting in random_settings: if setting in self.random_settings_limits: - min = self.random_settings_limits[setting][0] - if min and random_settings[setting] < min: - random_settings[setting] = min - max = self.random_settings_limits[setting][1] - if max and random_settings[setting] > max: - random_settings[setting] = max + min_value = self.random_settings_limits[setting][0] + if min_value and random_settings[setting] < min_value: + random_settings[setting] = min_value + max_value = self.random_settings_limits[setting][1] + if max_value and random_settings[setting] > max_value: + random_settings[setting] = max_value def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name @@ -2005,7 +2005,7 @@ class TestSuite: all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 + tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2052,7 +2052,7 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 self.sequential_tests = [] self.parallel_tests = [] From 5226792b1d8b4e110c63a813fb68c9dd65ea07b7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:48:06 +0000 Subject: [PATCH 050/409] Fix bad merge with master --- tests/clickhouse-test | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 84f33860484..c4124982442 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1163,6 +1163,9 @@ class TestCase: elif args.cloud and ("no-replicated-database" in tags): return FailureReason.REPLICATED_DB + elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: + return FailureReason.DISTRIBUTED_CACHE + elif args.cloud and self.name in suite.cloud_skip_list: return FailureReason.NOT_SUPPORTED_IN_CLOUD @@ -1246,6 +1249,11 @@ class TestCase: ): return FailureReason.SKIP + elif "no-flaky-check" in tags and ( + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + ): + return FailureReason.SKIP + elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -2274,7 +2282,6 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) - break except TimeoutError: break finally: @@ -3260,6 +3267,12 @@ def parse_args(): default=False, help="Run tests over s3 storage", ) + parser.add_argument( + "--distributed-cache", + action="store_true", + default=False, + help="Run tests with enabled distributed cache", + ) parser.add_argument( "--azure-blob-storage", action="store_true", From bb33dca38470aba044da06938cc96ca55166262d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:49:08 +0000 Subject: [PATCH 051/409] Fix unrelated changes --- tests/clickhouse-test | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c4124982442..72136404796 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -682,6 +682,7 @@ class FailureReason(enum.Enum): BUILD = "not running for current build" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" SHARED_MERGE_TREE = "no-shared-merge-tree" + DISTRIBUTED_CACHE = "distributed-cache" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1163,9 +1164,6 @@ class TestCase: elif args.cloud and ("no-replicated-database" in tags): return FailureReason.REPLICATED_DB - elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: - return FailureReason.DISTRIBUTED_CACHE - elif args.cloud and self.name in suite.cloud_skip_list: return FailureReason.NOT_SUPPORTED_IN_CLOUD @@ -1212,6 +1210,9 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB + elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: + return FailureReason.DISTRIBUTED_CACHE + elif ( tags and ("atomic-database" in tags) @@ -1250,7 +1251,7 @@ class TestCase: return FailureReason.SKIP elif "no-flaky-check" in tags and ( - 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) ): return FailureReason.SKIP From 71c06b40cbf65abda49579bf5ac08e46575c7d29 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 09:07:21 +0000 Subject: [PATCH 052/409] Avoid regexp --- docs/en/development/tests.md | 4 ++-- tests/clickhouse-test | 14 +++++++------- ...amic_merges_1_horizontal_compact_merge_tree.sql | 2 +- ...namic_merges_1_horizontal_compact_wide_tree.sql | 2 +- ...ynamic_merges_1_vertical_compact_merge_tree.sql | 2 +- ...7_dynamic_merges_1_vertical_wide_merge_tree.sql | 2 +- ...amic_merges_2_horizontal_compact_merge_tree.sql | 2 +- ...dynamic_merges_2_horizontal_wide_merge_tree.sql | 2 +- ...ynamic_merges_2_vertical_compact_merge_tree.sql | 2 +- ...7_dynamic_merges_2_vertical_wide_merge_tree.sql | 2 +- ...38_nested_dynamic_merges_compact_horizontal.sql | 2 +- ...3038_nested_dynamic_merges_compact_vertical.sql | 2 +- ...03038_nested_dynamic_merges_wide_horizontal.sql | 2 +- .../03038_nested_dynamic_merges_wide_vertical.sql | 2 +- 14 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index f0afa983fec..bc9f85ef323 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -100,14 +100,14 @@ For `.sh` tests limits are written as a comment on the line next to tags or on t ```bash #!/usr/bin/env bash # Tags: no-fasttest -# Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +# Random settings limits: max_block_size=(1000, 10000); index_granularity=(100, None) ``` For `.sql` tests tags are placed as a SQL comment in the line next to tags or in the first line: ```sql -- Tags: no-fasttest --- Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +-- Random settings limits: max_block_size=(1000, 10000); index_granularity=(100, None) SELECT 1 ``` diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 72136404796..e5378e8c7f3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1964,13 +1964,13 @@ class TestSuite: if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 - # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' - random_settings_limits = re.findall( - "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str - ) - random_settings_limits = { - pair[0]: make_tuple(pair[1]) for pair in random_settings_limits - } + # limits are specified in a form 'setting1=(min, max); setting2=(min,max); ...' + random_settings_limits = {} + for setting_and_limit in random_settings_limits_str.split(';'): + setting_and_limit = setting_and_limit.split('=') + random_settings_limits[setting_and_limit[0].strip()] = make_tuple( + setting_and_limit[1] + ) return random_settings_limits def is_shebang(line: str) -> bool: diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index 7c2e7c3d2be..46f1c78b255 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index aa62435188a..bf0c6ef0374 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index bfc7bb9d206..fb82369a7a3 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index 233667db0a7..c026bc04a56 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index 71c6841515a..7f1934091f2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index 94ae1d867f5..f1f387fae9d 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index 98ae230636a..cc11c454d38 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index f8f5bd5d9e1..ffb2aca8b35 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 13c1fd8b485..9ec4e4f949b 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index daa95071cdb..ed4de931841 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index dea7e7c0971..bd3c4b58a8f 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index bf1323f2ea9..81bcda5443d 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From 621f4bbf9e04b62628a9c053b3f39c6b8a67a52d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 6 Aug 2024 14:13:20 +0200 Subject: [PATCH 053/409] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index f815a21b6a1..bb062deaab0 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,6 +79,7 @@ static std::initializer_list Date: Tue, 6 Aug 2024 15:01:10 +0200 Subject: [PATCH 054/409] Fix pylint --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e5378e8c7f3..dea303ecdfb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1086,7 +1086,7 @@ class TestCase: self.random_settings_limits = ( suite.all_random_settings_limits[case] if case in suite.all_random_settings_limits - else dict() + else {} ) for tag in os.getenv("GLOBAL_TAGS", "").split(","): From 20cd84960efa48c7978607f92e58bc4b66187840 Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Tue, 6 Aug 2024 21:42:14 +0800 Subject: [PATCH 055/409] modify code style modify code style --- src/Parsers/ParserDeleteQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 65d593b6c1b..3503fbe9ad2 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -62,6 +62,7 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; + if (query->partition) query->children.push_back(query->partition); From d124de847b44344d9346c4d1b76ada03b31c58c8 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 16:06:59 +0000 Subject: [PATCH 056/409] Fix style --- tests/clickhouse-test | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index dea303ecdfb..c3b1d4d907c 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1959,15 +1959,17 @@ class TestSuite: ) -> Dict[str, Tuple[int, int]]: if not line.startswith(comment_sign): return {} - random_settings_limits_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 + random_settings_limits_str = line[len(comment_sign) :].lstrip() random_settings_limits_prefix = "Random settings limits:" if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} - random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 + random_settings_limits_str = random_settings_limits_str[ + len(random_settings_limits_prefix) : + ] # limits are specified in a form 'setting1=(min, max); setting2=(min,max); ...' random_settings_limits = {} - for setting_and_limit in random_settings_limits_str.split(';'): - setting_and_limit = setting_and_limit.split('=') + for setting_and_limit in random_settings_limits_str.split(";"): + setting_and_limit = setting_and_limit.split("=") random_settings_limits[setting_and_limit[0].strip()] = make_tuple( setting_and_limit[1] ) @@ -2014,7 +2016,12 @@ class TestSuite: all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 + ( + tags, + random_settings_limits, + ) = load_tags_and_random_settings_limits_from_file( + os.path.join(suite_dir, test_name) + ) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2061,7 +2068,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] From 0ebe8e35511f764b61cb2428433132644f7deb96 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 6 Aug 2024 18:38:23 +0200 Subject: [PATCH 057/409] Fix style --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e6949dd4fba..1ebc9b07748 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,7 +79,7 @@ static std::initializer_list Date: Wed, 7 Aug 2024 02:28:13 +0000 Subject: [PATCH 058/409] 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 6882e8ad79bcc1ce0b2c80e82aee38f57a4504bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 14:34:33 +0200 Subject: [PATCH 059/409] Revert "Merge pull request #67130 from rschu1ze/unflake-win-view-tests2" This reverts commit 48e61a295cf3e2a7ab93dc99531d14701b90004c, reversing changes made to da24aa06fac26bf1516320cc6e49c8927b1f600a. --- .../queries/0_stateless/01052_window_view_proc_tumble_to_now.sh | 1 - tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh | 1 - tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh | 1 - tests/queries/0_stateless/01055_window_view_proc_hop_to.sh | 1 - .../0_stateless/01075_window_view_proc_tumble_to_now_populate.sh | 1 - 5 files changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index c473bf766b0..a235b60cee9 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh index ca89dd9daf1..b4a647c9864 100755 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh index 67cac226de6..af9a47714b5 100755 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh index e44fc2e363c..5a7e92e6a1b 100755 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh index 859b9a86a2a..220bb39602b 100755 --- a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh +++ b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From cfeb20681d3af5f2a5d538096e5117be06e7c624 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 7 Aug 2024 14:42:42 +0200 Subject: [PATCH 060/409] Fix style check --- tests/clickhouse-test | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c3b1d4d907c..5946e561949 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2068,10 +2068,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] - + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( + all_tags_and_random_settings_limits[1] + ) self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 5f3417153439b56b68a34be890b75884941800dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 15:12:06 +0200 Subject: [PATCH 061/409] Fix window view missing blocks due to slow flush to view --- src/Storages/WindowView/StorageWindowView.cpp | 25 ++++++++++++++++--- .../01056_window_view_proc_hop_watch.py | 7 +++--- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5830c844582..94eed575ca8 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1051,17 +1051,27 @@ void StorageWindowView::threadFuncFireProc() if (shutdown_called) return; + /// Acquiring the lock can take seconds (depends on how long it takes to push) so we keep a reference to remember + /// what's the starting point where we want to push from + UInt32 timestamp_start = now(); + std::lock_guard lock(fire_signal_mutex); /// TODO: consider using time_t instead (for every timestamp in this class) UInt32 timestamp_now = now(); - LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}", timestamp_now, next_fire_signal, max_watermark); + LOG_TRACE( + log, + "Start: {}, now: {}, next fire signal: {}, max watermark: {}", + timestamp_start, + timestamp_now, + next_fire_signal, + max_watermark); while (next_fire_signal <= timestamp_now) { try { - if (max_watermark >= timestamp_now) + if (max_watermark >= timestamp_start) fire(next_fire_signal); } catch (...) @@ -1075,8 +1085,15 @@ void StorageWindowView::threadFuncFireProc() slide_interval *= 86400; next_fire_signal += slide_interval; - LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}", - timestamp_now, next_fire_signal, max_watermark, max_fired_watermark, slide_interval); + LOG_TRACE( + log, + "Start: {}, now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}", + timestamp_start, + timestamp_now, + next_fire_signal, + max_watermark, + max_fired_watermark, + slide_interval); } if (max_watermark >= timestamp_now) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 6bb8db2c38e..bc4ec16f1b1 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -20,7 +20,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.expect(prompt) - client1.send("SET enable_analyzer = 0") + client1.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") client1.expect(prompt) client1.send("SET allow_experimental_window_view = 1") client1.expect(prompt) @@ -28,7 +28,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) - client2.send("SET enable_analyzer = 0") + client2.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch") @@ -42,8 +42,9 @@ with client(name="client1>", log=log) as client1, client( "CREATE TABLE 01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" ) client1.expect(prompt) + # Introduce a sleep call to verify that even if the push to view is slow WATCH will work client1.send( - "CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;" + "CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(sleep(5)) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;" ) client1.expect(prompt) From aca7e6734ceb5a0ec7a8d6f8f3bfa5bde05c860e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 16:24:56 +0200 Subject: [PATCH 062/409] Style --- .../0_stateless/01056_window_view_proc_hop_watch.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index bc4ec16f1b1..e0f969050b5 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -20,7 +20,9 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.expect(prompt) - client1.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") + client1.send( + "SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000" + ) client1.expect(prompt) client1.send("SET allow_experimental_window_view = 1") client1.expect(prompt) @@ -28,7 +30,9 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) - client2.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") + client2.send( + "SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000" + ) client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch") From f5e993df2a3ed28505a6fdc22db16306c0308733 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 7 Aug 2024 16:46:33 +0000 Subject: [PATCH 063/409] Add tests --- .../DataLakes/IStorageDataLake.h | 49 +- .../StorageObjectStorageSource.cpp | 2 + .../registerDataLakeTableFunctions.cpp | 20 +- tests/integration/helpers/cloud_tools.py | 13 +- .../test_storage_azure_blob_storage/test.py | 2 + .../configs/config.d/named_collections.xml | 4 + .../integration/test_storage_iceberg/test.py | 424 +++++++++--------- 7 files changed, 291 insertions(+), 223 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..123a629f395 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,10 +48,20 @@ public: ConfigurationPtr configuration = base_configuration->clone(); + try { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - configuration->setPaths(metadata->getDataFiles()); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); + + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); + } + configuration->setPaths(data_files); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -80,9 +90,20 @@ public: const std::optional & format_settings_, ContextPtr local_context) { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); + auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); + + auto schema_from_metadata = metadata->getTableSchema(); + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); + } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -90,7 +111,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(metadata->getDataFiles()); + configuration->setPaths(data_files); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -99,14 +120,33 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); + if (current_metadata && *current_metadata == *new_metadata) return; + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); + + current_metadata = std::move(new_metadata); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); + + auto data_files = current_metadata->getDataFiles(); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); + + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); + } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(current_metadata->getDataFiles()); + updated_configuration->setPaths(data_files); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -148,7 +188,10 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); + current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index e760098f10f..399e4a56ba8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -438,6 +438,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 40561a1e075..8de852f4fba 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -17,13 +17,25 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); # endif # if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", + .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); # endif + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", + .examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); } #endif @@ -62,11 +74,13 @@ void registerTableFunctionHudi(TableFunctionFactory & factory) void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#if USE_PARQUET +#endif +#if USE_AWS_S3 +# if USE_PARQUET registerTableFunctionDeltaLake(factory); #endif registerTableFunctionHudi(factory); diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 534791b8bc5..0d009b5f3fe 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -52,15 +52,16 @@ class LocalUploader(CloudUploader): class AzureUploader(CloudUploader): - def __init__(self, blob_service_client, container_name): - self.blob_service_client = blob_service_client - self.container_name = container_name + + def __init__(self, container_client): + self.container_client = container_client def upload_file(self, local_path, remote_blob_path): - blob_client = self.blob_service_client.get_blob_client( - container=self.container_name, blob=remote_blob_path - ) + # print("Local path", local_path) + # print("Remote blob path", remote_blob_path) + blob_client = self.container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: + # print("Data", data) blob_client.upload_blob(data, overwrite=True) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6fbe7634642..c5599d20519 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,6 +550,7 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] + print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -559,6 +560,7 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' + assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index d4c54e2d13d..3d80fa2ecf9 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -5,5 +5,9 @@ minio minio123 + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7f83846bd89..b8a67600785 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -36,6 +36,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject +from azure.storage.blob import BlobServiceClient from helpers.cloud_tools import ( prepare_s3_bucket, @@ -92,12 +93,27 @@ def started_cluster(): cluster.minio_client, cluster.minio_bucket ) - container_name = "my_container" + cluster.azure_container_name = "mycontainer" - cluster.default_azurite_uploader = AzureUploader( - cluster.blob_service_client, container_name + # connection_string = ( + # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" + # ) + # local_blob_service_client = BlobServiceClient.from_connection_string( + # cluster.env_variables["AZURITE_CONNECTION_STRING"] + # ) + + local_blob_service_client = cluster.blob_service_client + + container_client = local_blob_service_client.create_container( + cluster.azure_container_name ) + cluster.container_client = container_client + + cluster.default_azure_uploader = AzureUploader(container_client) + yield cluster finally: @@ -162,10 +178,24 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + table_function=False, + **kwargs, +): if storage_type == "local": pass elif storage_type == "s3": + if "bucket" in kwargs: + bucket = kwargs["bucket"] + else: + bucket = cluster.minio_bucket + if table_function: + return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( f""" DROP TABLE IF EXISTS {table_name}; @@ -173,11 +203,15 @@ def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwa ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" ) elif storage_type == "azure": + if table_function: + return f""" + icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) + """ node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) else: raise Exception("Unknown iceberg storage type: {}", storage_type) @@ -218,11 +252,11 @@ def default_upload_directory(started_cluster, storage_type, local_path, remote_p @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -233,96 +267,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "", ) - # for bucket in minio_client.list_buckets(): - # for object in minio_client.list_objects(bucket.name, recursive=True): - # print("Object: ", object.object_name) - # extension = object.object_name.split(".")[-1] - # print("File extension: ", extension) - # try: - # response = minio_client.get_object( - # object.bucket_name, object.object_name - # ) - - # if extension == "avro": - # avro_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # avro_file = io.BytesIO(avro_bytes) - - # # Read the Avro data - # reader = avro.datafile.DataFileReader( - # avro_file, avro.io.DatumReader() - # ) - # records = [record for record in reader] - - # # Close the reader - # reader.close() - - # # Now you can work with the records - # for record in records: - # # print(json.dumps(record, indent=4, sort_keys=True)) - # print(str(record)) - # # my_json = ( - # # str(record) - # # .replace("'", '"') - # # .replace("None", "null") - # # .replace('b"', '"') - # # ) - # # print(my_json) - # # data = json.loads(my_json) - # # s = json.dumps(data, indent=4, sort_keys=True) - # # print(s) - # elif extension == "json": - # my_bytes_value = response.read() - # my_json = my_bytes_value.decode("utf8").replace("'", '"') - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - # elif extension == "parquet": - # # print("To be continued...") - # # # Your byte string containing the Parquet data - # # parquet_bytes = response.read() - - # # # Use BytesIO to create a file-like object from the byte string - # # parquet_file = io.BytesIO(parquet_bytes) - - # # # Read the Parquet data into a PyArrow Table - # # table = pq.read_table(parquet_file) - - # # # Convert the PyArrow Table to a Pandas DataFrame - # # df = table.to_pandas() - - # # # Now you can work with s DataFrame - # # print(df) - # parquet_bytes = ( - # response.read() - # ) # Replace with your actual byte string - - # # Create a temporary file and write the byte string to it - # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - # tmp_file.write(parquet_bytes) - # tmp_file_path = tmp_file.name - - # # Read the Parquet file using PySpark - # df = spark.read.parquet(tmp_file_path) - - # # Show the DataFrame - # print(df.toPandas()) - # else: - # print(response.read()) - - # finally: - # print("----------------") - # response.close() - # response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - # print("Debug Print") - - # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) - - # print(instance.query("SELECT number FROM numbers(100)")) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( "SELECT number, toString(number + 1) FROM numbers(100)" @@ -332,12 +277,11 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_partition_by(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_by_" + format_version + TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -348,22 +292,23 @@ def test_partition_by(started_cluster, format_version): partition_by="a", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_multiple_iceberg_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -373,9 +318,13 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) + # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/3127466b-299d-48ca-a367-6b9b1df1e78c-m0.avro', @@ -383,7 +332,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): # '/iceberg_data/default/test_multiple_iceberg_files/metadata/v1.metadata.json'] assert len(files) == 5 - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 write_iceberg_from_df( @@ -393,8 +342,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): mode="append", format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert len(files) == 9 @@ -405,12 +357,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_types(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_types_" + format_version + TABLE_NAME = "test_types_" + format_version + "_" + storage_type data = [ ( @@ -436,24 +387,29 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 assert ( instance.query(f"SELECT a, b, c, d, e FROM {TABLE_NAME}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function = f"iceberg(s3, filename='iceberg_data/default/{TABLE_NAME}/')" + table_function_expr = create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, table_function=True + ) assert ( - instance.query(f"SELECT a, b, c, d, e FROM {table_function}").strip() + instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - assert instance.query(f"DESCRIBE {table_function} FORMAT TSV") == TSV( + assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV( [ ["a", "Nullable(Int32)"], ["b", "Nullable(String)"], @@ -465,12 +421,13 @@ def test_types(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_delete_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -480,17 +437,22 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -504,27 +466,34 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_evolved_schema(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -534,19 +503,25 @@ def test_evolved_schema(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 expected_data = instance.query(f"SELECT * FROM {TABLE_NAME} order by a, b") spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -558,12 +533,13 @@ def test_evolved_schema(started_cluster, format_version): assert data == expected_data -def test_row_based_deletes(started_cluster): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes" + TABLE_NAME = "test_row_based_deletes_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -572,17 +548,23 @@ def test_row_based_deletes(started_cluster): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -590,13 +572,19 @@ def test_row_based_deletes(started_cluster): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_schema_inference(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket for format in ["Parquet", "ORC", "Avro"]: - TABLE_NAME = "test_schema_inference_" + format + "_" + format_version + TABLE_NAME = ( + "test_schema_inference_" + + format + + "_" + + format_version + + "_" + + storage_type + ) # Types time, timestamptz, fixed are not supported in Spark. spark.sql( @@ -606,12 +594,16 @@ def test_schema_inference(started_cluster, format_version): spark.sql( f"insert into {TABLE_NAME} select 42, 4242, 42.42, 4242.4242, decimal(42.42), decimal(42.42), decimal(42.42), date('2020-01-01'), timestamp('2020-01-01 20:00:00'), 'hello', binary('hello'), array(1,2,3), map('key', 'value'), struct(42, 'hello'), array(struct(map('key', array(map('key', 42))), struct(42, 'hello')))" ) - - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME, format) + create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, format=format + ) res = instance.query( f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0} @@ -646,12 +638,11 @@ def test_schema_inference(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_selection(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_" + format_version + TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -662,22 +653,26 @@ def test_metadata_file_selection(started_cluster, format_version): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_format_with_uuid(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_with_uuid_" + format_version + TABLE_NAME = ( + "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -694,63 +689,70 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -def test_restart_broken(started_cluster): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = "broken2" - TABLE_NAME = "test_restart_broken_table_function" +# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) +# def test_restart_broken(started_cluster): +# instance = started_cluster.instances["node1"] +# spark = started_cluster.spark_session +# minio_client = started_cluster.minio_client +# bucket = "broken2" +# TABLE_NAME = "test_restart_broken_table_function" - if not minio_client.bucket_exists(bucket): - minio_client.make_bucket(bucket) +# if not minio_client.bucket_exists(bucket): +# minio_client.make_bucket(bucket) - parquet_data_path = create_initial_data_file( - started_cluster, - instance, - "SELECT number, toString(number) FROM numbers(100)", - TABLE_NAME, - ) +# parquet_data_path = create_initial_data_file( +# started_cluster, +# instance, +# "SELECT number, toString(number) FROM numbers(100)", +# TABLE_NAME, +# ) - write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - create_iceberg_table(instance, TABLE_NAME, bucket=bucket) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") +# files = default_upload_directory( +# started_cluster, +# storage_type, +# f"/iceberg_data/default/{TABLE_NAME}/", +# "", +# ) +# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - s3_objects = list_s3_objects(minio_client, bucket, prefix="") - assert ( - len( - list( - minio_client.remove_objects( - bucket, - [DeleteObject(obj) for obj in s3_objects], - ) - ) - ) - == 0 - ) - minio_client.remove_bucket(bucket) +# s3_objects = list_s3_objects(minio_client, bucket, prefix="") +# assert ( +# len( +# list( +# minio_client.remove_objects( +# bucket, +# [DeleteObject(obj) for obj in s3_objects], +# ) +# ) +# ) +# == 0 +# ) +# minio_client.remove_bucket(bucket) - instance.restart_clickhouse() +# instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error( - f"SELECT count() FROM {TABLE_NAME}" - ) +# assert "NoSuchBucket" in instance.query_and_get_error( +# f"SELECT count() FROM {TABLE_NAME}" +# ) - minio_client.make_bucket(bucket) +# minio_client.make_bucket(bucket) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) +# files = default_upload_directory( +# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" +# ) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 22dad244e75137580e08aa2cc83538e5ffd950c8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 8 Aug 2024 01:50:49 +0000 Subject: [PATCH 064/409] 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 35eb4fa1766c6f5b5d81cbfc68e63cadd4b01838 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 12:03:14 +0200 Subject: [PATCH 065/409] Schedule WV cleanup after any fire trigger --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 94eed575ca8..4a20a07ae89 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1096,7 +1096,7 @@ void StorageWindowView::threadFuncFireProc() slide_interval); } - if (max_watermark >= timestamp_now) + if (max_watermark >= timestamp_start) clean_cache_task->schedule(); UInt64 next_fire_ms = static_cast(next_fire_signal) * 1000; From 4e97549f7336d33e4c3fc08fec21ce7af9f433a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 14:22:11 +0200 Subject: [PATCH 066/409] Make 02884_authentication_quota.sh more parallel --- .../0_stateless/02884_authentication_quota.sh | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02884_authentication_quota.sh b/tests/queries/0_stateless/02884_authentication_quota.sh index f013bb4d639..ef48ac741f0 100755 --- a/tests/queries/0_stateless/02884_authentication_quota.sh +++ b/tests/queries/0_stateless/02884_authentication_quota.sh @@ -1,13 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -QUOTA="2884_quota_$$" -USER="2884_user_$$" -ROLE="2884_role_$$" +QUOTA="2884_quota_${CLICKHOUSE_DATABASE}" +USER="2884_user_${CLICKHOUSE_DATABASE}" +ROLE="2884_role_${CLICKHOUSE_DATABASE}" function login_test() @@ -26,10 +25,10 @@ function login_test() echo "> Alter the quota with MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4" ${CLICKHOUSE_CLIENT} -q "ALTER QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4 TO ${USER}" - + echo "> Try to login to the user account with correct password" ${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null" - + echo "> Successfull login should reset failed authentications counter. Check the failed_sequential_authentications, max_failed_sequential_authentications fields." ${CLICKHOUSE_CLIENT} -q "SELECT failed_sequential_authentications, max_failed_sequential_authentications FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" @@ -39,7 +38,7 @@ function login_test() ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect' ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect' ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED' - + echo "> Also try to login with correct password. Quota should stay exceeded." ${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED' From dbf743d58d7b6100183b2409cf3259da1a1a457f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 14:49:35 +0200 Subject: [PATCH 067/409] Make 02722_database_filesystem parallelizable --- .../02722_database_filesystem.reference | 2 +- .../0_stateless/02722_database_filesystem.sh | 60 +++++++++---------- 2 files changed, 30 insertions(+), 32 deletions(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.reference b/tests/queries/0_stateless/02722_database_filesystem.reference index 2aa7e74befb..290bb9d4fbc 100644 --- a/tests/queries/0_stateless/02722_database_filesystem.reference +++ b/tests/queries/0_stateless/02722_database_filesystem.reference @@ -1,6 +1,6 @@ Test 1: create filesystem database and check implicit calls 0 -test1 +default_test1 4 4 30 diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index fa23d847d90..e28c9358f1e 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -15,14 +14,11 @@ echo '2,"def",456,"bacabaa"' >> ${user_files_tmp_dir}/tmp.csv echo '3,"story",78912,"acabaab"' >> ${user_files_tmp_dir}/tmp.csv echo '4,"history",21321321,"cabaaba"' >> ${user_files_tmp_dir}/tmp.csv -tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME} - $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_1.csv') select * from numbers(1, 10)" $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_2.csv') select * from numbers(11, 10)" $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_30.csv') select * from numbers(21, 10)" -[[ -d $tmp_dir ]] && rm -rd $tmp_dir -mkdir $tmp_dir +tmp_dir=$(mktemp -d ${CLICKHOUSE_TEST_UNIQUE_NAME}_XXXX) cp ${user_files_tmp_dir}/tmp.csv ${tmp_dir}/tmp.csv cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp/tmp.csv cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp.myext @@ -30,55 +26,57 @@ cp ${user_files_tmp_dir}/tmp_numbers_1.csv ${user_files_tmp_dir}/tmp/tmp_numbers ################# echo "Test 1: create filesystem database and check implicit calls" +DATABASE_TEST1="${CLICKHOUSE_DATABASE}_test1" ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test1; -CREATE DATABASE test1 ENGINE = Filesystem; +DROP DATABASE IF EXISTS ${DATABASE_TEST1}; +CREATE DATABASE ${DATABASE_TEST1} ENGINE = Filesystem; """ echo $? -${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/tmp.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/*/*tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "${DATABASE_TEST1}" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/*/*tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; # **/* does not search in the current directory but searches recursively in nested directories. -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" ################# echo "Test 2: check DatabaseFilesystem access rights and errors handling on server" # DATABASE_ACCESS_DENIED: Allows list files only inside user_files -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --multiline --query """ -USE test1; +USE ${DATABASE_TEST1}; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; """ 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: path should be inside user_files +DATABASE_TEST2="${CLICKHOUSE_DATABASE}_test2" ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test2; -CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); +DROP DATABASE IF EXISTS ${DATABASE_TEST2}; +CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('/tmp'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test2; -CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); +DROP DATABASE IF EXISTS ${DATABASE_TEST2}; +CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: # Clean -${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" +${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS ${DATABASE_TEST1};" rm -rd $tmp_dir rm -rd $user_files_tmp_dir From de0d6f036853e8c74f2d0bde3a754a924c294388 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 15:00:13 +0200 Subject: [PATCH 068/409] Make 03002_part_log_rmt_fetch_* tests parallel --- .../0_stateless/03002_part_log_rmt_fetch_merge_error.sh | 8 ++++---- .../0_stateless/03002_part_log_rmt_fetch_mutate_error.sh | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh index e58c542b8ac..dbaf7bf20c4 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest # SMT: The merge process is completely different from RMT +# no-fasttest: Avoid long waits CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,8 +13,7 @@ function wait_until() { local q=$1 && shift while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do - # too frequent FLUSH LOGS is too costly - sleep 2 + sleep 0.5 done } @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT -nm -q " system start replicated sends rmt_master; " # wait until rmt_slave will fetch the part and reflect this error in system.part_log -wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" $CLICKHOUSE_CLIENT -nm -q " system sync replica rmt_slave; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh index cc8f53aafb9..a6a02946602 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest # SMT: The merge process is completely different from RMT +# no-fasttest: Avoid long waits CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -15,8 +16,7 @@ function wait_until() { local q=$1 && shift while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do - # too frequent FLUSH LOGS is too costly - sleep 2 + sleep 0.5 done } @@ -43,9 +43,8 @@ $CLICKHOUSE_CLIENT -nm -q " wait_for_mutation rmt_master 0000000000 $CLICKHOUSE_CLIENT -nm -q "system start pulling replication log rmt_slave" # and wait until rmt_slave to fetch the part and reflect this error in system.part_log -wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" $CLICKHOUSE_CLIENT -nm -q " - system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; From d3dffce4409a6afb1968c09a4f79ce21fb861515 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 15:03:46 +0200 Subject: [PATCH 069/409] Make 02247_written_bytes_quota parallel --- .../0_stateless/02247_written_bytes_quota.sh | 51 ++++++++++--------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/02247_written_bytes_quota.sh b/tests/queries/0_stateless/02247_written_bytes_quota.sh index 2430184c4df..ce32c76ab18 100755 --- a/tests/queries/0_stateless/02247_written_bytes_quota.sh +++ b/tests/queries/0_stateless/02247_written_bytes_quota.sh @@ -1,48 +1,51 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +ROLE="r_${CLICKHOUSE_TEST_UNIQUE_NAME}" +USER="u_${CLICKHOUSE_TEST_UNIQUE_NAME}" +QUOTA="q_${CLICKHOUSE_TEST_UNIQUE_NAME}" + ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247" -${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE written_bytes_02247(s String) ENGINE = Memory" -${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02247" -${CLICKHOUSE_CLIENT} -q "CREATE USER u02247" -${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247" -${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247" -${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO r02247" +${CLICKHOUSE_CLIENT} -q "CREATE ROLE ${ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${USER}" +${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO ${ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT ${ROLE} to ${USER}" +${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO ${ROLE}" # The value 'qwqw' means about 13 bytes are to be written, so the current quota (30 bytes) gives the ability to write 'qwqw' 2 times. -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" -#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA" -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" -#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA" -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" +#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA" +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" +#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA" +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = 'q02247'" +${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA q02247" -${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA ${QUOTA}" +${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO ${ROLE}" ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247" # Numbers from 0 to 50 means about 540 bytes are to be written, so the current quota (1000 bytes) is enough to do so. -${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)" +${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)" # Numbers from 0 to 100 means about 1090 bytes are to be written, so the current quota (1000 bytes total - 540 bytes already used) is NOT enough to do so. -${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'" +${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247" -${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}" From 4b7080f2cd4881fa2c369e4a05a3904744d23867 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 16:49:33 +0200 Subject: [PATCH 070/409] Parallelize 01600_detach_permanently --- tests/queries/0_stateless/01600_detach_permanently.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index be405f8e7d9..4810d2e9732 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-fasttest +# no-fasttest: It's a bit demanding # Creation of a database with Ordinary engine emits a warning. CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal @@ -9,9 +10,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ## tests with real clickhouse restart would be a bit to heavy, -## to ensure the table will not reappear back clickhose-local is enough. +## to ensure the table will not reappear back clickhouse-local is enough. -WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/01600_detach_permanently" +WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/${CLICKHOUSE_TEST_UNIQUE_NAME}" rm -rf "${WORKING_FOLDER_01600}" mkdir -p "${WORKING_FOLDER_01600}" From 453b62b692a9aa2a623e737a43002f1534182a43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 16:53:07 +0200 Subject: [PATCH 071/409] 02455_one_row_from_csv_memory_usage is parallelizable --- .../queries/0_stateless/02455_one_row_from_csv_memory_usage.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 7906f2917c4..1a0bc69f2e8 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz +ln -s "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz ${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=1000000000" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=100000000" From 09627d9a0910279eece78f4471142dbbf6048f7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 17:01:38 +0200 Subject: [PATCH 072/409] Make 01038_dictionary_lifetime_min_zero_sec parallelizable --- .../0_stateless/01038_dictionary_lifetime_min_zero_sec.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh index 66732205f95..8e15453cbe5 100755 --- a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh +++ b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -42,6 +41,7 @@ function check() while [ "$query_result" != "2.2" ] do + sleep 0.2 query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))") done } From a53050d45455bb5159f752527dc40ec0ef53d6bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 17:01:46 +0200 Subject: [PATCH 073/409] Add some comments --- .../01360_materialized_view_with_join_on_query_log.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index 1b22082bd83..b9ad132bcd2 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +-- no-parallel: Slows down query_log DROP TABLE IF EXISTS slow_log; DROP TABLE IF EXISTS expected_times; From f2731841de804c30ece1c75e84c8ca8d3eb62ef8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 20:20:21 +0200 Subject: [PATCH 074/409] init --- src/Core/callOnTypeIndex.h | 3 + src/DataTypes/getLeastSupertype.cpp | 41 +++++ src/DataTypes/getLeastSupertype.h | 22 +++ src/Functions/FunctionsConversion.cpp | 61 +++++++- ...23_interval_data_type_comparison.reference | 99 ++++++++++++ .../03223_interval_data_type_comparison.sql | 142 ++++++++++++++++++ 6 files changed, 365 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03223_interval_data_type_comparison.reference create mode 100644 tests/queries/0_stateless/03223_interval_data_type_comparison.sql diff --git a/src/Core/callOnTypeIndex.h b/src/Core/callOnTypeIndex.h index f5f67df563b..ae5afce36be 100644 --- a/src/Core/callOnTypeIndex.h +++ b/src/Core/callOnTypeIndex.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -212,6 +213,8 @@ static bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... ar case TypeIndex::IPv4: return f(TypePair(), std::forward(args)...); case TypeIndex::IPv6: return f(TypePair(), std::forward(args)...); + case TypeIndex::Interval: return f(TypePair(), std::forward(args)...); + default: break; } diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index a71b19d6c92..0b9c744c091 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -228,6 +228,40 @@ void convertUInt64toInt64IfPossible(const DataTypes & types, TypeIndexSet & type } } +DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet &types_set) +{ + const auto& granularity_map = getGranularityMap(); + int min_granularity = std::get<0>(granularity_map.at(IntervalKind::Kind::Year)); + DataTypePtr smallest_type; + + bool is_higher_interval = false; // For Years, Quarters and Months + + for (const auto &type : types) + { + if (const auto * interval_type = typeid_cast(type.get())) + { + int current_granularity = std::get<0>(granularity_map.at(interval_type->getKind())); + if (current_granularity > 8) + is_higher_interval = true; + if (current_granularity < min_granularity) + { + min_granularity = current_granularity; + smallest_type = type; + } + } + } + + if (is_higher_interval && min_granularity <= 8) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + + if (smallest_type) + { + types_set.clear(); + types_set.insert(smallest_type->getTypeId()); + } + + return smallest_type; +} } template @@ -652,6 +686,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return numeric_type; } + /// For interval data types. + { + auto res = findSmallestIntervalSuperType(types, type_ids); + if (res) + return res; + } + /// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases). return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); } diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 2ae1e52ca96..c584eb83011 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -1,5 +1,7 @@ #pragma once #include +#include +#include namespace DB { @@ -48,4 +50,24 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); +/// A map that enumerated all interval kinds in ascending order with a conversion value to a next interval +inline const std::unordered_map> & getGranularityMap() +{ + static std::unordered_map> granularity_map = + { + {IntervalKind::Kind::Nanosecond, {1, 1000}}, + {IntervalKind::Kind::Microsecond, {2, 1000}}, + {IntervalKind::Kind::Millisecond, {3, 1000}}, + {IntervalKind::Kind::Second, {4, 60}}, + {IntervalKind::Kind::Minute, {5, 60}}, + {IntervalKind::Kind::Hour, {6, 24}}, + {IntervalKind::Kind::Day, {7, 7}}, + {IntervalKind::Kind::Week, {8, 4}}, + {IntervalKind::Kind::Month, {9, 3}}, + {IntervalKind::Kind::Quarter, {10, 4}}, + {IntervalKind::Kind::Year, {11, 1}} + }; + return granularity_map; +} + } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..0ab1858dc97 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -1573,6 +1574,55 @@ struct ConvertImpl arguments, result_type, input_rows_count, additions); } } + else if constexpr (std::is_same_v && std::is_same_v) + { + IntervalKind to = typeid_cast(result_type.get())->getKind(); + IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); + + if (from == to) + return arguments[0].column; + + const auto &map = getGranularityMap(); + Int64 conversion_factor = 1; + Int64 result_value; + + int from_position = map.at(from).first; + int to_position = map.at(to).first; // Positions of each interval according to granurality map + + if (from_position < to_position) + { + for (int i = from_position - 1; i <= to_position; ++i) + { + // Find the kind that matches this position + for (const auto &entry : map) + { + if (entry.second.first == i) + { + conversion_factor *= entry.second.second; + break; + } + } + } + result_value = arguments[0].column->getInt(0) / conversion_factor; + } + else + { + for (int i = from_position - 1; i >= to_position; --i) + { + for (const auto &entry : map) + { + if (entry.second.first == i) + { + conversion_factor *= entry.second.second; + break; + } + } + } + result_value = arguments[0].column->getInt(0) * conversion_factor; + } + + return ColumnConst::create(ColumnInt64::create(1, result_value), input_rows_count); + } else { using FromFieldType = typename FromDataType::FieldType; @@ -2181,7 +2231,7 @@ private: const DataTypePtr from_type = removeNullable(arguments[0].type); ColumnPtr result_column; - [[maybe_unused]] FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; if (context) date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value; @@ -2277,7 +2327,7 @@ private: } } else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); return true; }; @@ -2334,6 +2384,11 @@ private: else done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); } + + if constexpr (std::is_same_v) + { + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); + } } if (!done) @@ -5224,7 +5279,7 @@ REGISTER_FUNCTION(Conversion) /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); + factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.reference b/tests/queries/0_stateless/03223_interval_data_type_comparison.reference new file mode 100644 index 00000000000..e98f792e4b2 --- /dev/null +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.reference @@ -0,0 +1,99 @@ +Comparing nanoseconds +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +Comparing microseconds +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +Comparing milliseconds +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +Comparing seconds +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +Comparing minutes +1 +1 +1 +1 +0 +0 +0 +0 +Comparing hours +1 +1 +1 +0 +0 +0 +Comparing days +1 +1 +0 +0 +Comparing weeks +1 +0 +Comparing months +1 +1 +1 +0 +0 +0 +Comparing quarters +1 +1 +0 +0 +Comparing years +1 +0 diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql new file mode 100644 index 00000000000..6e4862bf2d2 --- /dev/null +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -0,0 +1,142 @@ +SELECT('Comparing nanoseconds'); +SELECT toIntervalNanosecond(500) > toIntervalNanosecond(300); +SELECT toIntervalNanosecond(1000) < toIntervalNanosecond(1500); +SELECT toIntervalNanosecond(2000) = toIntervalNanosecond(2000); +SELECT toIntervalNanosecond(1000) >= toIntervalMicrosecond(1); +SELECT toIntervalNanosecond(1000001) > toIntervalMillisecond(1); +SELECT toIntervalNanosecond(2000000001) > toIntervalSecond(2); +SELECT toIntervalNanosecond(60000000000) = toIntervalMinute(1); +SELECT toIntervalNanosecond(7199999999999) < toIntervalHour(2); +SELECT toIntervalNanosecond(1) < toIntervalDay(2); +SELECT toIntervalNanosecond(5) < toIntervalWeek(1); + +SELECT toIntervalNanosecond(500) < toIntervalNanosecond(300); +SELECT toIntervalNanosecond(1000) > toIntervalNanosecond(1500); +SELECT toIntervalNanosecond(2000) != toIntervalNanosecond(2000); +SELECT toIntervalNanosecond(1000) < toIntervalMicrosecond(1); +SELECT toIntervalNanosecond(1000001) < toIntervalMillisecond(1); +SELECT toIntervalNanosecond(2000000001) < toIntervalSecond(2); +SELECT toIntervalNanosecond(60000000000) != toIntervalMinute(1); +SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); +SELECT toIntervalNanosecond(1) > toIntervalDay(2); +SELECT toIntervalNanosecond(5) > toIntervalWeek(1); + +SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing microseconds'); +SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); +SELECT toIntervalMicrosecond(1001) > toIntervalMillisecond(1); +SELECT toIntervalMicrosecond(2000000) = toIntervalSecond(2); +SELECT toIntervalMicrosecond(179999999) < toIntervalMinute(3); +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); +SELECT toIntervalMicrosecond(36000000000000) > toIntervalDay(2); +SELECT toIntervalMicrosecond(1209600000000) = toIntervalWeek(2); + +SELECT toIntervalMicrosecond(1) > toIntervalMicrosecond(999); +SELECT toIntervalMicrosecond(1001) < toIntervalMillisecond(1); +SELECT toIntervalMicrosecond(2000000) != toIntervalSecond(2); +SELECT toIntervalMicrosecond(179999999) > toIntervalMinute(3); +SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); +SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); +SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); + +SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing milliseconds'); +SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); +SELECT toIntervalMillisecond(2000) = toIntervalSecond(2); +SELECT toIntervalMillisecond(170000) < toIntervalMinute(3); +SELECT toIntervalMillisecond(144000001) > toIntervalHour(40); +SELECT toIntervalMillisecond(1728000000) = toIntervalDay(20); +SELECT toIntervalMillisecond(1198599999) < toIntervalWeek(2); + +SELECT toIntervalMillisecond(2000) < toIntervalMillisecond(2); +SELECT toIntervalMillisecond(2000) != toIntervalSecond(2); +SELECT toIntervalMillisecond(170000) > toIntervalMinute(3); +SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); +SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); +SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); + +SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing seconds'); +SELECT toIntervalSecond(120) > toIntervalSecond(2); +SELECT toIntervalSecond(120) = toIntervalMinute(2); +SELECT toIntervalSecond(1) < toIntervalHour(2); +SELECT toIntervalSecond(86401) >= toIntervalDay(1); +SELECT toIntervalSecond(1209600) = toIntervalWeek(2); + +SELECT toIntervalSecond(120) < toIntervalSecond(2); +SELECT toIntervalSecond(120) != toIntervalMinute(2); +SELECT toIntervalSecond(1) > toIntervalHour(2); +SELECT toIntervalSecond(86401) < toIntervalDay(1); +SELECT toIntervalSecond(1209600) != toIntervalWeek(2); + +SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing minutes'); +SELECT toIntervalMinute(1) < toIntervalMinute(59); +SELECT toIntervalMinute(1) < toIntervalHour(59); +SELECT toIntervalMinute(1440) = toIntervalDay(1); +SELECT toIntervalMinute(30241) > toIntervalWeek(3); + +SELECT toIntervalMinute(1) > toIntervalMinute(59); +SELECT toIntervalMinute(1) > toIntervalHour(59); +SELECT toIntervalMinute(1440) != toIntervalDay(1); +SELECT toIntervalMinute(30241) < toIntervalWeek(3); + +SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing hours'); +SELECT toIntervalHour(48) > toIntervalHour(2); +SELECT toIntervalHour(48) >= toIntervalDay(2); +SELECT toIntervalHour(672) = toIntervalWeek(4); + +SELECT toIntervalHour(48) < toIntervalHour(2); +SELECT toIntervalHour(48) < toIntervalDay(2); +SELECT toIntervalHour(672) != toIntervalWeek(4); + +SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing days'); +SELECT toIntervalDay(1) < toIntervalDay(23); +SELECT toIntervalDay(25) > toIntervalWeek(3); + +SELECT toIntervalDay(1) > toIntervalDay(23); +SELECT toIntervalDay(25) < toIntervalWeek(3); + +SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing weeks'); +SELECT toIntervalWeek(1) < toIntervalWeek(6); + +SELECT toIntervalWeek(1) > toIntervalWeek(6); + +SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing months'); +SELECT toIntervalMonth(1) < toIntervalMonth(3); +SELECT toIntervalMonth(124) > toIntervalQuarter(5); +SELECT toIntervalMonth(36) = toIntervalYear(3); + +SELECT toIntervalMonth(1) > toIntervalMonth(3); +SELECT toIntervalMonth(124) < toIntervalQuarter(5); +SELECT toIntervalMonth(36) != toIntervalYear(3); + +SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing quarters'); +SELECT toIntervalQuarter(5) > toIntervalQuarter(4); +SELECT toIntervalQuarter(20) = toIntervalYear(5); + +SELECT toIntervalQuarter(5) < toIntervalQuarter(4); +SELECT toIntervalQuarter(20) != toIntervalYear(5); + +SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing years'); +SELECT toIntervalYear(1) < toIntervalYear(3); + +SELECT toIntervalYear(1) > toIntervalYear(3); + +SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From e9659626adc29d237d23e0f3ced9c8712d472a73 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 20:41:15 +0200 Subject: [PATCH 075/409] fix style + add docs --- .../data-types/special-data-types/interval.md | 21 +++++++++---------- .../data-types/special-data-types/interval.md | 21 +++++++++---------- .../data-types/special-data-types/interval.md | 21 +++++++++---------- src/DataTypes/getLeastSupertype.cpp | 2 +- 4 files changed, 31 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/data-types/special-data-types/interval.md b/docs/en/sql-reference/data-types/special-data-types/interval.md index bedbcf0bd28..be26053580b 100644 --- a/docs/en/sql-reference/data-types/special-data-types/interval.md +++ b/docs/en/sql-reference/data-types/special-data-types/interval.md @@ -53,29 +53,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -Intervals with different types can’t be combined. You can’t use intervals like `4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, the interval `1 day and an hour` interval can be expressed as `25 HOUR` or `90000 SECOND`. - -You can’t perform arithmetical operations with `Interval`-type values, but you can add intervals of different types consequently to values in `Date` or `DateTime` data types. For example: +Also it is possible to use multiple intervals simultaneously: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┐ +│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -The following query causes an exception: +And to compare values with different intevals: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┐ +│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## See Also diff --git a/docs/ru/sql-reference/data-types/special-data-types/interval.md b/docs/ru/sql-reference/data-types/special-data-types/interval.md index 867a6665f4b..5064391f582 100644 --- a/docs/ru/sql-reference/data-types/special-data-types/interval.md +++ b/docs/ru/sql-reference/data-types/special-data-types/interval.md @@ -54,29 +54,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -Нельзя объединять интервалы различных типов. Нельзя использовать интервалы вида `4 DAY 1 HOUR`. Вместо этого выражайте интервал в единицах меньших или равных минимальной единице интервала, например, интервал «1 день и 1 час» можно выразить как `25 HOUR` или `90000 SECOND`. - -Арифметические операции со значениями типов `Interval` не доступны, однако можно последовательно добавлять различные интервалы к значениям типов `Date` и `DateTime`. Например: +Также можно использовать различные типы интервалов одновременно: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┐ +│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -Следующий запрос приведёт к генерированию исключения: +И сравнивать значения из разными интервалами: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┐ +│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## Смотрите также {#smotrite-takzhe} diff --git a/docs/zh/sql-reference/data-types/special-data-types/interval.md b/docs/zh/sql-reference/data-types/special-data-types/interval.md index e05869b2df8..e16f6d5f84f 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/interval.md +++ b/docs/zh/sql-reference/data-types/special-data-types/interval.md @@ -55,29 +55,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -不同类型的间隔不能合并。 你不能使用诸如 `4 DAY 1 HOUR` 的时间间隔. 以小于或等于时间间隔最小单位的单位来指定间隔,例如,时间间隔 `1 day and an hour` 可以表示为 `25 HOUR` 或 `90000 SECOND`. - -你不能对 `Interval` 类型的值执行算术运算,但你可以向 `Date` 或 `DateTime` 数据类型的值添加不同类型的时间间隔,例如: +也可以同時使用多個間隔: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┐ +│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -以下查询将导致异常: +並比較不同直數的值: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┐ +│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## 另请参阅 {#see-also} diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 0b9c744c091..674284460dc 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -252,7 +252,7 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & } if (is_higher_interval && min_granularity <= 8) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); if (smallest_type) { From 0ad6aa09acb72a67fc88e0cd8186afd32fefd6bf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 22:51:27 +0200 Subject: [PATCH 076/409] fix style --- docs/en/sql-reference/data-types/special-data-types/interval.md | 2 +- src/Functions/FunctionsConversion.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/special-data-types/interval.md b/docs/en/sql-reference/data-types/special-data-types/interval.md index be26053580b..4ef1a7e6238 100644 --- a/docs/en/sql-reference/data-types/special-data-types/interval.md +++ b/docs/en/sql-reference/data-types/special-data-types/interval.md @@ -65,7 +65,7 @@ SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVA └─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -And to compare values with different intevals: +And to compare values with different intervals: ``` sql SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 0ab1858dc97..1708991af74 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1606,7 +1606,7 @@ struct ConvertImpl result_value = arguments[0].column->getInt(0) / conversion_factor; } else - { + { for (int i = from_position - 1; i >= to_position; --i) { for (const auto &entry : map) From 94efbb0bf9ab62a5399d4918e7bcfd358421a879 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 23:26:24 +0200 Subject: [PATCH 077/409] fix build --- src/Functions/FunctionsConversion.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 1708991af74..43ebe573582 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -5279,7 +5279,7 @@ REGISTER_FUNCTION(Conversion) /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive); + factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); From b4c553718353eb2302f85ea4d096a92036ce832c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 23:49:56 +0200 Subject: [PATCH 078/409] fix errorcodes in test --- .../03223_interval_data_type_comparison.sql | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql index 6e4862bf2d2..5d01addae45 100644 --- a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -21,7 +21,7 @@ SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); SELECT toIntervalNanosecond(1) > toIntervalDay(2); SELECT toIntervalNanosecond(5) > toIntervalWeek(1); -SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError NO_COMMON_TYPE } SELECT('Comparing microseconds'); SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); @@ -40,7 +40,7 @@ SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); -SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing milliseconds'); SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); @@ -57,7 +57,7 @@ SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); -SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing seconds'); SELECT toIntervalSecond(120) > toIntervalSecond(2); @@ -72,7 +72,7 @@ SELECT toIntervalSecond(1) > toIntervalHour(2); SELECT toIntervalSecond(86401) < toIntervalDay(1); SELECT toIntervalSecond(1209600) != toIntervalWeek(2); -SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing minutes'); SELECT toIntervalMinute(1) < toIntervalMinute(59); @@ -85,7 +85,7 @@ SELECT toIntervalMinute(1) > toIntervalHour(59); SELECT toIntervalMinute(1440) != toIntervalDay(1); SELECT toIntervalMinute(30241) < toIntervalWeek(3); -SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError NO_COMMON_TYPE } SELECT('Comparing hours'); SELECT toIntervalHour(48) > toIntervalHour(2); @@ -96,7 +96,7 @@ SELECT toIntervalHour(48) < toIntervalHour(2); SELECT toIntervalHour(48) < toIntervalDay(2); SELECT toIntervalHour(672) != toIntervalWeek(4); -SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing days'); SELECT toIntervalDay(1) < toIntervalDay(23); @@ -105,14 +105,14 @@ SELECT toIntervalDay(25) > toIntervalWeek(3); SELECT toIntervalDay(1) > toIntervalDay(23); SELECT toIntervalDay(25) < toIntervalWeek(3); -SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError NO_COMMON_TYPE } SELECT('Comparing weeks'); SELECT toIntervalWeek(1) < toIntervalWeek(6); SELECT toIntervalWeek(1) > toIntervalWeek(6); -SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError NO_COMMON_TYPE } SELECT('Comparing months'); SELECT toIntervalMonth(1) < toIntervalMonth(3); @@ -123,7 +123,7 @@ SELECT toIntervalMonth(1) > toIntervalMonth(3); SELECT toIntervalMonth(124) < toIntervalQuarter(5); SELECT toIntervalMonth(36) != toIntervalYear(3); -SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError NO_COMMON_TYPE } SELECT('Comparing quarters'); SELECT toIntervalQuarter(5) > toIntervalQuarter(4); @@ -132,11 +132,11 @@ SELECT toIntervalQuarter(20) = toIntervalYear(5); SELECT toIntervalQuarter(5) < toIntervalQuarter(4); SELECT toIntervalQuarter(20) != toIntervalYear(5); -SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError NO_COMMON_TYPE } SELECT('Comparing years'); SELECT toIntervalYear(1) < toIntervalYear(3); SELECT toIntervalYear(1) > toIntervalYear(3); -SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError NO_COMMON_TYPE } From 29ce915d00f6a48a7dfa6a70ec9889d47eacf584 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Aug 2024 09:37:16 +0000 Subject: [PATCH 079/409] Try to fix bug --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 ++ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 ++ src/Disks/IO/createReadBufferFromFileBase.cpp | 28 +++++++++++++------ .../Local/LocalObjectStorage.cpp | 9 +++--- src/IO/ReadBuffer.h | 3 ++ .../StorageObjectStorageSource.cpp | 5 +--- 6 files changed, 33 insertions(+), 16 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index bb9761a3905..80dcc1baa14 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,6 +169,7 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { + // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); @@ -230,6 +231,7 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; + // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 9f1cb681f1a..a8d5bd1797e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,6 +85,8 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; + std::deque buffer_cemetery_; + LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index b132e25ac6b..c6152543a41 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,14 +1,15 @@ +#include +#include #include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include #include +#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -77,6 +78,7 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -88,6 +90,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); + res = std::make_unique( filename, buffer_size, @@ -99,6 +103,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); + #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -117,6 +123,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); + auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -131,6 +139,7 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -144,8 +153,11 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + } return res; }; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index c37b47bbc93..d291dcd65cf 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -48,11 +48,12 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); + auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); + return kek; }; return std::make_unique( diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 73f5335411f..ea41aab84f0 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -103,6 +103,7 @@ public: */ bool ALWAYS_INLINE eof() { + LOG_DEBUG(); return !hasPendingData() && !next(); } @@ -182,6 +183,8 @@ public: while (bytes_copied < n && !eof()) { + auto k = *pos; + LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 399e4a56ba8..9233986d858 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -413,10 +413,7 @@ std::future StorageObjectStorageSource } std::unique_ptr StorageObjectStorageSource::createReadBuffer( - const ObjectInfo & object_info, - const ObjectStoragePtr & object_storage, - const ContextPtr & context_, - const LoggerPtr & log) + const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; From 3357275fa8c55bcc5371b4ff9c9a5d80e51ab689 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 18:33:45 +0800 Subject: [PATCH 080/409] Fix MSAN issue caused by incorrect date format. --- src/IO/ReadHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..dd4aef23a25 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1432,7 +1432,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, date_broken_down_length + 1 + size)); else return false; } From 35f19522e745ef2267b4c6f99dfc5d7c1f7e78c3 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 9 Aug 2024 12:56:14 +0200 Subject: [PATCH 081/409] fix fuzzer --- src/Functions/FunctionsConversion.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 43ebe573582..c25bc44450f 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1579,7 +1579,7 @@ struct ConvertImpl IntervalKind to = typeid_cast(result_type.get())->getKind(); IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); - if (from == to) + if (from == to || arguments[0].column->empty()) return arguments[0].column; const auto &map = getGranularityMap(); From ca4041847e4aa8acccd6ea31c0a18f2160c0dc7a Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 19:15:41 +0800 Subject: [PATCH 082/409] Add tests --- src/IO/ReadHelpers.cpp | 4 ++-- ...215_fix_datetime_implicit_conversion.reference | 1 + .../03215_fix_datetime_implicit_conversion.sql | 15 +++++++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference create mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index dd4aef23a25..e69b4187b37 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1402,7 +1402,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", String(s, already_read_length)); else return false; } @@ -1432,7 +1432,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, date_broken_down_length + 1 + size)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, size)); else return false; } diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql new file mode 100644 index 00000000000..70a8a3432a6 --- /dev/null +++ b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS tab SYNC; + +CREATE TABLE tab +( + a DateTime, + pk String +) Engine = MergeTree() ORDER BY pk; + +INSERT INTO tab select cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 1; + +SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'; +SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } +SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'; -- { serverError TYPE_MISMATCH } + +DROP TABLE IF EXISTS tab SYNC; From c3ab8266ebf86da96712acf97ee0efe9e924d777 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rodolphe=20Dug=C3=A9=20de=20Bernonville?= Date: Fri, 9 Aug 2024 14:05:18 +0200 Subject: [PATCH 083/409] odbc: get http retry from server configuration --- src/BridgeHelper/XDBCBridgeHelper.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 5f4c7fd8381..0630a0f24e3 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -100,6 +100,7 @@ protected: auto buf = BuilderRWBufferFromHTTP(getPingURI()) .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); return checkString(PING_OK_ANSWER, *buf); @@ -206,6 +207,7 @@ protected: .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); bool res = false; @@ -232,6 +234,7 @@ protected: .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); std::string character; From 6ded5e1c8b994ad2332468e605b17a74e8d5675f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 23:50:03 +0800 Subject: [PATCH 084/409] Some fixups --- src/IO/ReadHelpers.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index e69b4187b37..b484f80250d 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1399,10 +1399,8 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D size_t size = buf.read(s_pos, remaining_date_size); if (size != remaining_date_size) { - s_pos[size] = 0; - if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", String(s, already_read_length)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", std::string_view(s, already_read_length + size)); else return false; } @@ -1429,10 +1427,8 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (size != time_broken_down_length) { - s_pos[size] = 0; - if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, size)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", std::string_view(s, size)); else return false; } From a3d8db6e1eb27d6a8fa81bbf43c8ffb171714c0b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 9 Aug 2024 19:05:37 +0200 Subject: [PATCH 085/409] updates due to review --- .../data-types/special-data-types/interval.md | 21 +++++++------- src/DataTypes/getLeastSupertype.cpp | 13 ++++----- src/DataTypes/getLeastSupertype.h | 21 ++------------ src/Functions/FunctionsConversion.cpp | 28 +++---------------- 4 files changed, 24 insertions(+), 59 deletions(-) diff --git a/docs/zh/sql-reference/data-types/special-data-types/interval.md b/docs/zh/sql-reference/data-types/special-data-types/interval.md index e16f6d5f84f..e05869b2df8 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/interval.md +++ b/docs/zh/sql-reference/data-types/special-data-types/interval.md @@ -55,28 +55,29 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -也可以同時使用多個間隔: +不同类型的间隔不能合并。 你不能使用诸如 `4 DAY 1 HOUR` 的时间间隔. 以小于或等于时间间隔最小单位的单位来指定间隔,例如,时间间隔 `1 day and an hour` 可以表示为 `25 HOUR` 或 `90000 SECOND`. + +你不能对 `Interval` 类型的值执行算术运算,但你可以向 `Date` 或 `DateTime` 数据类型的值添加不同类型的时间间隔,例如: ``` sql -SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR ``` ``` text -┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┐ -│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ -└─────────────────────┴────────────────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ +│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ +└─────────────────────┴────────────────────────────────────────────────────────┘ ``` -並比較不同直數的值: +以下查询将导致异常: ``` sql -SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); +select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┐ -│ 1 │ -└─────────────────────────────────────────────────────────────┘ +Received exception from server (version 19.14.1): +Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. ``` ## 另请参阅 {#see-also} diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 674284460dc..8bcec49815f 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -230,8 +230,7 @@ void convertUInt64toInt64IfPossible(const DataTypes & types, TypeIndexSet & type DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet &types_set) { - const auto& granularity_map = getGranularityMap(); - int min_granularity = std::get<0>(granularity_map.at(IntervalKind::Kind::Year)); + auto min_interval = IntervalKind::Kind::Year; DataTypePtr smallest_type; bool is_higher_interval = false; // For Years, Quarters and Months @@ -240,18 +239,18 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & { if (const auto * interval_type = typeid_cast(type.get())) { - int current_granularity = std::get<0>(granularity_map.at(interval_type->getKind())); - if (current_granularity > 8) + auto current_interval = interval_type->getKind().kind; + if (current_interval > IntervalKind::Kind::Week) is_higher_interval = true; - if (current_granularity < min_granularity) + if (current_interval < min_interval) { - min_granularity = current_granularity; + min_interval = current_interval; smallest_type = type; } } } - if (is_higher_interval && min_granularity <= 8) + if (is_higher_interval && min_interval <= IntervalKind::Kind::Week) throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); if (smallest_type) diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index c584eb83011..5ea2b6417b2 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -50,24 +50,9 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); -/// A map that enumerated all interval kinds in ascending order with a conversion value to a next interval -inline const std::unordered_map> & getGranularityMap() -{ - static std::unordered_map> granularity_map = - { - {IntervalKind::Kind::Nanosecond, {1, 1000}}, - {IntervalKind::Kind::Microsecond, {2, 1000}}, - {IntervalKind::Kind::Millisecond, {3, 1000}}, - {IntervalKind::Kind::Second, {4, 60}}, - {IntervalKind::Kind::Minute, {5, 60}}, - {IntervalKind::Kind::Hour, {6, 24}}, - {IntervalKind::Kind::Day, {7, 7}}, - {IntervalKind::Kind::Week, {8, 4}}, - {IntervalKind::Kind::Month, {9, 3}}, - {IntervalKind::Kind::Quarter, {10, 4}}, - {IntervalKind::Kind::Year, {11, 1}} - }; - return granularity_map; +/// A vector that shows the conversion rates to the next Interval type starting from NanoSecond +static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; + } } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index c25bc44450f..25c6bbcbfef 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1582,42 +1582,22 @@ struct ConvertImpl if (from == to || arguments[0].column->empty()) return arguments[0].column; - const auto &map = getGranularityMap(); Int64 conversion_factor = 1; Int64 result_value; - int from_position = map.at(from).first; - int to_position = map.at(to).first; // Positions of each interval according to granurality map + int from_position = static_cast(from.kind); + int to_position = static_cast(to.kind); // Positions of each interval according to granurality map if (from_position < to_position) { for (int i = from_position - 1; i <= to_position; ++i) - { - // Find the kind that matches this position - for (const auto &entry : map) - { - if (entry.second.first == i) - { - conversion_factor *= entry.second.second; - break; - } - } - } + conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) / conversion_factor; } else { for (int i = from_position - 1; i >= to_position; --i) - { - for (const auto &entry : map) - { - if (entry.second.first == i) - { - conversion_factor *= entry.second.second; - break; - } - } - } + conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) * conversion_factor; } From 384aedccaeece56456ad1e5ea17a8da4f56a69a4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sat, 10 Aug 2024 00:09:50 +0200 Subject: [PATCH 086/409] Update getLeastSupertype.h --- src/DataTypes/getLeastSupertype.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 5ea2b6417b2..8dd1685e6e9 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -54,5 +54,3 @@ DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; } - -} From a837df164c48f0b8041b122adc8fa80a148629f8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 02:37:42 +0000 Subject: [PATCH 087/409] 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 bc5d7933824d535bfbf2c672e5d7405c8bde86c8 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 09:13:24 +0000 Subject: [PATCH 088/409] Replace asyncronouos buffer with syncronouos --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/createReadBufferFromFileBase.cpp | 9 --------- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 11 +++-------- src/IO/ReadBuffer.h | 8 +------- 4 files changed, 4 insertions(+), 25 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 80dcc1baa14..01eb3d86696 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -231,7 +231,6 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; - // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index c6152543a41..76588bcb5b1 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -78,7 +78,6 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -90,8 +89,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); - res = std::make_unique( filename, buffer_size, @@ -103,7 +100,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); @@ -123,8 +119,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); - auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -139,7 +133,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -154,8 +147,6 @@ std::unique_ptr createReadBufferFromFileBase( } else { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); } return res; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index d291dcd65cf..3b650adb71f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -43,18 +43,13 @@ bool LocalObjectStorage::exists(const StoredObject & object) const std::unique_ptr LocalObjectStorage::readObjects( /// NOLINT const StoredObjects & objects, const ReadSettings & read_settings, - std::optional read_hint, - std::optional file_size) const + std::optional, + std::optional) const { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr - { - LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); - auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); - return kek; - }; + { return std::make_unique(object.remote_path); }; return std::make_unique( std::move(read_buffer_creator), diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index ea41aab84f0..98f0c8c9059 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,11 +101,7 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() - { - LOG_DEBUG(); - return !hasPendingData() && !next(); - } + bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } void ignore() { @@ -183,8 +179,6 @@ public: while (bytes_copied < n && !eof()) { - auto k = *pos; - LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; From bbf10088869b0490fb81957b3417c7776f2c3089 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:15:54 +0000 Subject: [PATCH 089/409] Fix test --- tests/integration/helpers/cloud_tools.py | 59 ++++-- tests/integration/helpers/cluster.py | 1 + .../configs/config.d/named_collections.xml | 2 + .../integration/test_storage_iceberg/test.py | 194 ++++++++++-------- 4 files changed, 150 insertions(+), 106 deletions(-) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 0d009b5f3fe..39c56e0eb85 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -9,7 +9,9 @@ from enum import Enum class CloudUploader: - def upload_directory(self, local_path, remote_blob_path): + + def upload_directory(self, local_path, remote_blob_path, **kwargs): + print(kwargs) result_files = [] # print(f"Arguments: {local_path}, {s3_path}") # for local_file in glob.glob(local_path + "/**"): @@ -18,12 +20,11 @@ class CloudUploader: result_local_path = os.path.join(local_path, local_file) result_remote_blob_path = os.path.join(remote_blob_path, local_file) if os.path.isfile(local_file): - self.upload_file(result_local_path, result_remote_blob_path) + self.upload_file(result_local_path, result_remote_blob_path, **kwargs) result_files.append(result_remote_blob_path) else: files = self.upload_directory( - result_local_path, - result_remote_blob_path, + result_local_path, result_remote_blob_path, **kwargs ) result_files.extend(files) return result_files @@ -34,37 +35,61 @@ class S3Uploader(CloudUploader): self.minio_client = minio_client self.bucket_name = bucket_name - def upload_file(self, local_path, remote_blob_path): + def upload_file(self, local_path, remote_blob_path, bucket=None): + print(f"Upload to bucket: {bucket}") + if bucket is None: + bucket = self.bucket_name self.minio_client.fput_object( - bucket_name=self.bucket_name, + bucket_name=bucket, object_name=remote_blob_path, file_path=local_path, ) class LocalUploader(CloudUploader): - def __init__(self): - pass + + def __init__(self, clickhouse_node): + self.clickhouse_node = clickhouse_node def upload_file(self, local_path, remote_blob_path): - if local_path != remote_blob_path: - shutil.copyfile(local_path, remote_blob_path) + dir_path = os.path.dirname(remote_blob_path) + if dir_path != "": + self.clickhouse_node.exec_in_container( + [ + "bash", + "-c", + "mkdir -p {}".format(dir_path), + ] + ) + self.clickhouse_node.copy_file_to_container(local_path, remote_blob_path) class AzureUploader(CloudUploader): - def __init__(self, container_client): - self.container_client = container_client + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_client = self.blob_service_client.get_container_client( + container_name + ) - def upload_file(self, local_path, remote_blob_path): - # print("Local path", local_path) - # print("Remote blob path", remote_blob_path) - blob_client = self.container_client.get_blob_client(remote_blob_path) + def upload_file(self, local_path, remote_blob_path, container_name=None): + if container_name is None: + container_client = self.container_client + else: + container_client = self.blob_service_client.get_container_client( + container_name + ) + blob_client = container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: - # print("Data", data) blob_client.upload_blob(data, overwrite=True) +def upload_directory(minio_client, bucket, local_path, remote_path): + S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + local_path, remote_path + ) + + def get_file_contents(minio_client, bucket, s3_path): data = minio_client.get_object(bucket, s3_path) data_str = b"" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 215718463e8..7e85bcf2f3c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,6 +2204,7 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") + self.exec_in_container( container_id, [ diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index 3d80fa2ecf9..b488638dd19 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -9,5 +9,7 @@ devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b8a67600785..b1d9b7f66bf 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -95,24 +95,19 @@ def started_cluster(): cluster.azure_container_name = "mycontainer" - # connection_string = ( - # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" - # ) - # local_blob_service_client = BlobServiceClient.from_connection_string( - # cluster.env_variables["AZURITE_CONNECTION_STRING"] - # ) + cluster.blob_service_client = cluster.blob_service_client - local_blob_service_client = cluster.blob_service_client - - container_client = local_blob_service_client.create_container( + container_client = cluster.blob_service_client.create_container( cluster.azure_container_name ) cluster.container_client = container_client - cluster.default_azure_uploader = AzureUploader(container_client) + cluster.default_azure_uploader = AzureUploader( + cluster.blob_service_client, cluster.azure_container_name + ) + + cluster.default_local_uploader = LocalUploader(cluster.instances["node1"]) yield cluster @@ -187,13 +182,12 @@ def create_iceberg_table( table_function=False, **kwargs, ): - if storage_type == "local": - pass - elif storage_type == "s3": + if storage_type == "s3": if "bucket" in kwargs: bucket = kwargs["bucket"] else: bucket = cluster.minio_bucket + print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( @@ -213,8 +207,19 @@ def create_iceberg_table( CREATE TABLE {table_name} ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) + elif storage_type == "local": + if table_function: + return f""" + icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) + """ + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" + ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") def create_initial_data_file( @@ -236,23 +241,28 @@ def create_initial_data_file( return result_path -def default_upload_directory(started_cluster, storage_type, local_path, remote_path): +def default_upload_directory( + started_cluster, storage_type, local_path, remote_path, **kwargs +): if storage_type == "local": - return LocalUploader().upload_directory(local_path, remote_path) + return started_cluster.default_local_uploader.upload_directory( + local_path, remote_path, **kwargs + ) elif storage_type == "s3": + print(kwargs) return started_cluster.default_s3_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) elif storage_type == "azure": return started_cluster.default_azure_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -264,7 +274,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -273,11 +283,9 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "SELECT number, toString(number + 1) FROM numbers(100)" ) - # assert 0 == 1 - @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -293,7 +301,10 @@ def test_partition_by(started_cluster, format_version, storage_type): ) files = default_upload_directory( - started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -302,7 +313,7 @@ def test_partition_by(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -322,7 +333,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', @@ -357,7 +368,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -391,7 +402,7 @@ def test_types(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -421,7 +432,7 @@ def test_types(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -441,7 +452,7 @@ def test_delete_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -487,7 +498,7 @@ def test_delete_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -507,7 +518,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -533,7 +544,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): assert data == expected_data -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -552,7 +563,7 @@ def test_row_based_deletes(started_cluster, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -572,7 +583,7 @@ def test_row_based_deletes(started_cluster, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -598,7 +609,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table( @@ -638,7 +649,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -657,7 +668,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -666,7 +677,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -693,7 +704,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -701,58 +712,63 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) -# def test_restart_broken(started_cluster): -# instance = started_cluster.instances["node1"] -# spark = started_cluster.spark_session -# minio_client = started_cluster.minio_client -# bucket = "broken2" -# TABLE_NAME = "test_restart_broken_table_function" +def test_restart_broken_s3(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + TABLE_NAME = "test_restart_broken_table_function_s3" -# if not minio_client.bucket_exists(bucket): -# minio_client.make_bucket(bucket) + minio_client = started_cluster.minio_client + bucket = "broken2" -# parquet_data_path = create_initial_data_file( -# started_cluster, -# instance, -# "SELECT number, toString(number) FROM numbers(100)", -# TABLE_NAME, -# ) + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) -# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") -# files = default_upload_directory( -# started_cluster, -# storage_type, -# f"/iceberg_data/default/{TABLE_NAME}/", -# "", -# ) -# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version="1", + ) -# s3_objects = list_s3_objects(minio_client, bucket, prefix="") -# assert ( -# len( -# list( -# minio_client.remove_objects( -# bucket, -# [DeleteObject(obj) for obj in s3_objects], -# ) -# ) -# ) -# == 0 -# ) -# minio_client.remove_bucket(bucket) + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + create_iceberg_table("s3", instance, TABLE_NAME, started_cluster, bucket=bucket) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 -# instance.restart_clickhouse() + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) -# assert "NoSuchBucket" in instance.query_and_get_error( -# f"SELECT count() FROM {TABLE_NAME}" -# ) + instance.restart_clickhouse() -# minio_client.make_bucket(bucket) + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) -# files = default_upload_directory( -# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" -# ) + minio_client.make_bucket(bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 0810703d6b162135a6f076e96052c433ef5b25d7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:38:38 +0000 Subject: [PATCH 090/409] Roll out strange changes --- .../test/integration/runner/requirements.txt | 1 - src/Core/SettingsChangesHistory.cpp | 271 +----------------- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 - src/Disks/IO/createReadBufferFromFileBase.cpp | 19 +- src/IO/ReadBuffer.h | 5 +- .../DataLakes/IStorageDataLake.h | 49 +--- .../helpers/{cloud_tools.py => s3_tools.py} | 0 .../test_iceberg_azure_storage/test.py | 2 +- .../test_local_storage/__init__.py | 0 .../test_local_storage/configs/config.xml | 3 - .../test_local_storage/files/example2.csv | 3 - tests/integration/test_local_storage/test.py | 148 ---------- tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 18 files changed, 26 insertions(+), 503 deletions(-) rename tests/integration/helpers/{cloud_tools.py => s3_tools.py} (100%) delete mode 100644 tests/integration/test_local_storage/__init__.py delete mode 100644 tests/integration/test_local_storage/configs/config.xml delete mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/test_local_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8bbb6a798ad..428986b5562 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,7 +64,6 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 -pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6637989202c..8f73e10c44f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -335,16 +75,7 @@ static std::initializer_listgetOrSet( - cache_key, file_offset_of_buffer_end, size, file_size.value(), - create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); } - return !file_segments->empty(); } @@ -161,8 +158,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current read range: [{}, {})", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); + "Having {} file segments to read: {}, current offset: {}", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); initialized = true; } @@ -1046,10 +1043,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; - - LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", - remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); - size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1062,8 +1055,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); + fmt::format("Expected {} <= {} (size: {}, read range: {})", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); } swap(*implementation_buffer); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 01eb3d86696..bb9761a3905 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,7 +169,6 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { - // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index a8d5bd1797e..9f1cb681f1a 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,8 +85,6 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; - std::deque buffer_cemetery_; - LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 76588bcb5b1..b132e25ac6b 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,15 +1,14 @@ -#include -#include #include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include #include -#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -100,7 +99,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -146,9 +144,8 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); - } + return res; }; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 98f0c8c9059..73f5335411f 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,7 +101,10 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } + bool ALWAYS_INLINE eof() + { + return !hasPendingData() && !next(); + } void ignore() { diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 123a629f395..c8603fccb86 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,20 +48,10 @@ public: ConfigurationPtr configuration = base_configuration->clone(); - try { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); - - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); - } - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -90,20 +80,9 @@ public: const std::optional & format_settings_, ContextPtr local_context) { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); - auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); - - auto schema_from_metadata = metadata->getTableSchema(); - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); - } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -111,7 +90,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -120,33 +99,14 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); - if (current_metadata && *current_metadata == *new_metadata) return; - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); - - current_metadata = std::move(new_metadata); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); - - auto data_files = current_metadata->getDataFiles(); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); - - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); - } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(data_files); + updated_configuration->setPaths(current_metadata->getDataFiles()); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -188,10 +148,7 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); - current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/s3_tools.py similarity index 100% rename from tests/integration/helpers/cloud_tools.py rename to tests/integration/helpers/s3_tools.py diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py index f96f8acfaaf..0b7179c3cc4 100644 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -37,7 +37,7 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from tests.integration.helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml deleted file mode 100644 index b4179ee51df..00000000000 --- a/tests/integration/test_local_storage/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - /var/lib/clickhouse/ - diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv deleted file mode 100644 index 7b6e6d6bab1..00000000000 --- a/tests/integration/test_local_storage/files/example2.csv +++ /dev/null @@ -1,3 +0,0 @@ -id,data -1,Str1 -2,Str2 diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py deleted file mode 100644 index 89c3c17e1f0..00000000000 --- a/tests/integration/test_local_storage/test.py +++ /dev/null @@ -1,148 +0,0 @@ -import logging -import os -import random -import string - -import pytest - -from helpers.cluster import ClickHouseCluster - -from pathlib import Path - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True - ) - - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def process_result(line: str): - return sorted( - list( - map( - lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), - filter(lambda x: len(x) > 0, line.split("\n")), - ) - ) - ) - - -def test_local_engine(started_cluster): - node = started_cluster.instances["test_local_storage"] - node.query( - """ - CREATE TABLE test_0 ( - id Int64, - data String - ) ENGINE=Local('/data/example.csv', 'CSV'); - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """ - ) - - result = node.query( - """ - select * from test_0; - """ - ) - - assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) - - error_got = node.query_and_get_error( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """ - ) - - node.query( - """ - SET engine_file_truncate_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """, - settings={"engine_file_truncate_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - node.query( - """ - SET local_create_new_file_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """, - settings={"local_create_new_file_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [ - (-1, "7"), - (0, "15"), - (1, "3"), - (4, "abc"), - (5, "arr"), - (9, "ty"), - ] == process_result(result) - - node.restart_clickhouse() - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - -def test_table_function(started_cluster): - node = started_cluster.instances["test_local_storage"] - - node.copy_file_to_container( - "test_local_storage/files/example2.csv", "/data/example2.csv" - ) - - result = node.query( - """ - SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); - """ - ) - - print("Res5", result) - - assert [(1, "Str1"), (2, "Str2")] == process_result(result) - - # assert False diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 1c1a7decdc3..35fe9d1bd6e 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 750b77b29f3..95825c2cd17 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents +from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b1d9b7f66bf..013a016efcd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -38,7 +38,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 31b1584507d..1c590824fd9 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.cloud_tools import prepare_s3_bucket +from tests.integration.helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 411e8f7cfb73d39df0614611766c3946603979ea Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:44:22 +0000 Subject: [PATCH 091/409] Remove unnecessary changes --- tests/integration/helpers/cluster.py | 1 - .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 - .../configs/users.d/users.xml | 9 - .../test_iceberg_azure_storage/test.py | 291 ------------------ .../test_storage_azure_blob_storage/test.py | 2 - tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 9 files changed, 3 insertions(+), 315 deletions(-) delete mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7e85bcf2f3c..215718463e8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,7 +2204,6 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") - self.exec_in_container( container_id, [ diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml deleted file mode 100644 index d4c54e2d13d..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - http://minio1:9001/root/ - minio - minio123 - - - diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py deleted file mode 100644 index 0b7179c3cc4..00000000000 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ /dev/null @@ -1,291 +0,0 @@ -import helpers.client -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.test_tools import TSV - -import pyspark -import logging -import os -import json -import pytest -import time -import glob -import uuid -import os - -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - -from pyspark.sql.types import ( - StructType, - StructField, - StringType, - IntegerType, - DateType, - TimestampType, - BooleanType, - ArrayType, -) -from pyspark.sql.functions import current_timestamp -from datetime import datetime -from pyspark.sql.functions import monotonically_increasing_id, row_number -from pyspark.sql.window import Window -from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 -from minio.deleteobjects import DeleteObject - -from tests.integration.helpers.s3_tools import ( - prepare_s3_bucket, - upload_directory, - get_file_contents, - list_s3_objects, -) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -def get_spark(): - builder = ( - pyspark.sql.SparkSession.builder.appName("spark_test") - .config( - "spark.sql.catalog.spark_catalog", - "org.apache.iceberg.spark.SparkSessionCatalog", - ) - .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") - .config("spark.sql.catalog.spark_catalog.type", "hadoop") - .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") - .config( - "spark.sql.extensions", - "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", - ) - .master("local") - ) - return builder.master("local").getOrCreate() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__, with_spark=True) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/named_collections.xml"], - user_configs=["configs/users.d/users.xml"], - with_minio=True, - stay_alive=True, - ) - - logging.info("Starting cluster...") - cluster.start() - - prepare_s3_bucket(cluster) - logging.info("S3 bucket created") - - cluster.spark_session = get_spark() - - yield cluster - - finally: - cluster.shutdown() - - -def run_query(instance, query, stdin=None, settings=None): - # type: (ClickHouseInstance, str, object, dict) -> str - - logging.info("Running query '{}'...".format(query)) - result = instance.query(query, stdin=stdin, settings=settings) - logging.info("Query finished") - - return result - - -def write_iceberg_from_file( - spark, path, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( - partition_by - ).tableProperty("format-version", format_version).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).append() - - -def write_iceberg_from_df( - spark, df, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).partitionedBy(partition_by).using("iceberg").create() - else: - df.writeTo(table_name).append() - - -def generate_data(spark, start, end): - a = spark.range(start, end, 1).toDF("a") - b = spark.range(start + 1, end + 1, 1).toDF("b") - b = b.withColumn("b", b["b"].cast(StringType())) - - a = a.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - b = b.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - - df = a.join(b, on=["row_index"]).drop("row_index") - return df - - -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) - - -def create_initial_data_file( - cluster, node, query, table_name, compression_method="none" -): - node.query( - f""" - INSERT INTO TABLE FUNCTION - file('{table_name}.parquet') - SETTINGS - output_format_parquet_compression_method='{compression_method}', - s3_truncate_on_insert=1 {query} - FORMAT Parquet""" - ) - user_files_path = os.path.join( - SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" - ) - result_path = f"{user_files_path}/{table_name}.parquet" - return result_path - - -@pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_single_iceberg_file_" + format_version - - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME - ) - - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) - - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - - for bucket in minio_client.list_buckets(): - for object in minio_client.list_objects(bucket.name, recursive=True): - print("Object: ", object.object_name) - extension = object.object_name.split(".")[-1] - print("File extension: ", extension) - try: - response = minio_client.get_object( - object.bucket_name, object.object_name - ) - - if extension == "avro": - avro_bytes = response.read() - - # Use BytesIO to create a file-like object from the byte string - avro_file = io.BytesIO(avro_bytes) - - # Read the Avro data - reader = avro.datafile.DataFileReader( - avro_file, avro.io.DatumReader() - ) - records = [record for record in reader] - - # Close the reader - reader.close() - - # Now you can work with the records - for record in records: - # print(json.dumps(record, indent=4, sort_keys=True)) - print(str(record)) - # my_json = ( - # str(record) - # .replace("'", '"') - # .replace("None", "null") - # .replace('b"', '"') - # ) - # print(my_json) - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - elif extension == "json": - my_bytes_value = response.read() - my_json = my_bytes_value.decode("utf8").replace("'", '"') - data = json.loads(my_json) - s = json.dumps(data, indent=4, sort_keys=True) - print(s) - elif extension == "parquet": - # print("To be continued...") - # # Your byte string containing the Parquet data - # parquet_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # parquet_file = io.BytesIO(parquet_bytes) - - # # Read the Parquet data into a PyArrow Table - # table = pq.read_table(parquet_file) - - # # Convert the PyArrow Table to a Pandas DataFrame - # df = table.to_pandas() - - # # Now you can work with s DataFrame - # print(df) - parquet_bytes = ( - response.read() - ) # Replace with your actual byte string - - # Create a temporary file and write the byte string to it - with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - tmp_file.write(parquet_bytes) - tmp_file_path = tmp_file.name - - # Read the Parquet file using PySpark - df = spark.read.parquet(tmp_file_path) - - # Show the DataFrame - print(df.toPandas()) - else: - print(response.read()) - - finally: - print("----------------") - response.close() - response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data - ) - - assert 0 == 1 diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index c5599d20519..6fbe7634642 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,7 +550,6 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] - print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -560,7 +559,6 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' - assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 35fe9d1bd6e..054b79ff6fe 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 95825c2cd17..0c3fbfb3cda 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1c590824fd9..ab327afe90b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from tests.integration.helpers.s3_tools import prepare_s3_bucket +from helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 42d20f2a8d34640233c2dc4002d5fe611f2f9c77 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:52:11 +0000 Subject: [PATCH 092/409] Remove table registration --- .../StorageObjectStorageSource.cpp | 2 -- .../registerStorageObjectStorage.cpp | 26 ++++--------------- .../TableFunctionObjectStorage.cpp | 17 ++++-------- .../TableFunctionObjectStorage.h | 24 ++++------------- 4 files changed, 15 insertions(+), 54 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 74fcd0418ab..ed68dd61967 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -437,8 +437,6 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); - auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 4bc8cfa6a2f..b5f4cf5bb54 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,15 +1,16 @@ #include #include #include -#include -#include #include +#include #include #include namespace DB { +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS + namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -69,6 +70,8 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } +#endif + #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { @@ -136,27 +139,8 @@ void registerStorageHDFS(StorageFactory & factory) } #endif -void registerStorageLocal(StorageFactory & factory) -{ - factory.registerStorage( - "Local", - [=](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - return createStorageObjectStorage(args, configuration, args.getLocalContext()); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::FILE, - }); -} - - void registerStorageObjectStorage(StorageFactory & factory) { - registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index d156afb81a1..550d9cc799b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,13 +12,12 @@ #include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -204,11 +203,6 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif - factory.registerFunction>( - {.documentation - = {.description = R"(The table function can be used to read the data stored locally.)", - .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, - .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -229,5 +223,4 @@ template class TableFunctionObjectStorage; #endif -template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 3468e5c5007..86b8f0d5e14 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include -#include -#include -#include -#include #include "config.h" +#include +#include +#include +#include +#include namespace DB { @@ -14,7 +14,6 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; -class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -91,17 +90,6 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; -struct LocalDefinition -{ - static constexpr auto name = "local"; - static constexpr auto storage_type_name = "Local"; - static constexpr auto signature = " - path\n" - " - path, format\n" - " - path, format, structure\n" - " - path, format, structure, compression_method\n"; - static constexpr auto max_number_of_arguments = 4; -}; - template class TableFunctionObjectStorage : public ITableFunction { @@ -181,6 +169,4 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif - -using TableFunctionLocal = TableFunctionObjectStorage; } From 58dd01d8b61e76c378a45bc84967b990bf355e4e Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 13:45:00 +0000 Subject: [PATCH 093/409] Add docs --- .../table-engines/integrations/iceberg.md | 30 +++++++++++----- .../sql-reference/table-functions/iceberg.md | 34 +++++++++++-------- .../integration/test_storage_iceberg/test.py | 10 +----- 3 files changed, 42 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 21fdbc0b1a5..40f3e7c22fd 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -6,28 +6,34 @@ sidebar_label: Iceberg # Iceberg Table Engine -This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure and locally stored tables. ## Create Table -Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table. +Note that the Iceberg table must already exist in the storage, this command does not take DDL parameters to create a new table. ``` sql -CREATE TABLE iceberg_table - ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,]) +CREATE TABLE iceberg_table_s3 + ENGINE = IcebergS3(url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression]) + +CREATE TABLE iceberg_table_azure + ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + +CREATE TABLE iceberg_table_local + ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) ``` -**Engine parameters** +**Engine arguments** -- `url` — url with the path to an existing Iceberg table. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. +Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +`format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +CREATE TABLE iceberg_table ENGINE=IcebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` Using named collections: @@ -45,9 +51,15 @@ Using named collections: ``` ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table') +CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table') + ``` +**Aliases** + + +Table engine `Iceberg` is an alias to `IcebergS3` now. + ## See also - [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index fa86b436a5e..dfe8d6f2d0a 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -6,35 +6,37 @@ sidebar_label: iceberg # iceberg Table Function -Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure or locally stored. ## Syntax ``` sql -iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) +icebergS3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method]) +icebergS3(named_collection[, option=value [,..]]) + +icebergAzure(connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method]) +icebergAzure(named_collection[, option=value [,..]]) + +icebergLocal(path_to_table, [,format] [,compression_method]) +icebergLocal(named_collection[, option=value [,..]]) ``` ## Arguments -- `url` — Bucket url with the path to an existing Iceberg table in S3. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). -- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - -Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). +Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +`format` stands for the format of data files in the Iceberg table. **Returned value** - -A table with the specified structure for reading data in the specified Iceberg table in S3. +A table with the specified structure for reading data in the specified Iceberg table. **Example** ```sql -SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` :::important -ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine. +ClickHouse currently supports reading v1 and v2 of the Iceberg format via the `icebergS3`, `icebergAzure` and `icebergLocal` table functions and `IcebergS3`, `icebergAzure` ans `icebergLocal` table engines. ::: ## Defining a named collection @@ -56,10 +58,14 @@ Here is an example of configuring a named collection for storing the URL and cre ``` ```sql -SELECT * FROM iceberg(iceberg_conf, filename = 'test_table') -DESCRIBE iceberg(iceberg_conf, filename = 'test_table') +SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table') +DESCRIBE icebergS3(iceberg_conf, filename = 'test_table') ``` +**Aliases** + +Table function `iceberg` is an alias to `icebergS3` now. + **See Also** - [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 013a016efcd..b66133f5562 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,14 +12,6 @@ import glob import uuid import os -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - from pyspark.sql.types import ( StructType, StructField, @@ -38,7 +30,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, From 61e616204eee76289e3da63dd663ce39a0316d0c Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 14:21:42 +0000 Subject: [PATCH 094/409] Docs correct typos --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- docs/en/sql-reference/table-functions/iceberg.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 40f3e7c22fd..78e076a7427 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -25,7 +25,7 @@ CREATE TABLE iceberg_table_local **Engine arguments** -Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +Description of the arguments coincides with description of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. `format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index dfe8d6f2d0a..784fd646860 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -23,7 +23,7 @@ icebergLocal(named_collection[, option=value [,..]]) ## Arguments -Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +Description of the arguments coincides with description of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. `format` stands for the format of data files in the Iceberg table. **Returned value** From 1cc845726842f388c4524d55b248f210e28d979d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 16:57:47 +0200 Subject: [PATCH 095/409] fix reviews, fix crash in fuzzer --- src/DataTypes/getLeastSupertype.cpp | 2 +- src/DataTypes/getLeastSupertype.h | 2 +- src/Functions/FunctionsConversion.cpp | 11 +- .../03223_interval_data_type_comparison.sql | 198 +++++++++--------- 4 files changed, 106 insertions(+), 107 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 8bcec49815f..65df529e78b 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -251,7 +251,7 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & } if (is_higher_interval && min_interval <= IntervalKind::Kind::Week) - throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the number of days in a month is not fixed", types[0]->getName(), types[1]->getName()); if (smallest_type) { diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 8dd1685e6e9..55d8e8fff0d 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -51,6 +51,6 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); /// A vector that shows the conversion rates to the next Interval type starting from NanoSecond -static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; +static std::vector interval_conversions = {1, 1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4}; } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 25c6bbcbfef..b6102cb7ecf 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1586,17 +1586,17 @@ struct ConvertImpl Int64 result_value; int from_position = static_cast(from.kind); - int to_position = static_cast(to.kind); // Positions of each interval according to granurality map + int to_position = static_cast(to.kind); /// Positions of each interval according to granularity map if (from_position < to_position) { - for (int i = from_position - 1; i <= to_position; ++i) + for (int i = from_position; i < to_position; ++i) conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) / conversion_factor; } else { - for (int i = from_position - 1; i >= to_position; --i) + for (int i = from_position; i > to_position; --i) conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) * conversion_factor; } @@ -2366,9 +2366,8 @@ private: } if constexpr (std::is_same_v) - { - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); - } + if (WhichDataType(from_type).isInterval()) + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); } if (!done) diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql index 5d01addae45..77b6e2fa3dc 100644 --- a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -1,142 +1,142 @@ SELECT('Comparing nanoseconds'); -SELECT toIntervalNanosecond(500) > toIntervalNanosecond(300); -SELECT toIntervalNanosecond(1000) < toIntervalNanosecond(1500); -SELECT toIntervalNanosecond(2000) = toIntervalNanosecond(2000); -SELECT toIntervalNanosecond(1000) >= toIntervalMicrosecond(1); -SELECT toIntervalNanosecond(1000001) > toIntervalMillisecond(1); -SELECT toIntervalNanosecond(2000000001) > toIntervalSecond(2); -SELECT toIntervalNanosecond(60000000000) = toIntervalMinute(1); -SELECT toIntervalNanosecond(7199999999999) < toIntervalHour(2); -SELECT toIntervalNanosecond(1) < toIntervalDay(2); -SELECT toIntervalNanosecond(5) < toIntervalWeek(1); +SELECT INTERVAL 500 NANOSECOND > INTERVAL 300 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND < INTERVAL 1500 NANOSECOND; +SELECT INTERVAL 2000 NANOSECOND = INTERVAL 2000 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND >= INTERVAL 1 MICROSECOND; +SELECT INTERVAL 1000001 NANOSECOND > INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000001 NANOSECOND > INTERVAL 2 SECOND; +SELECT INTERVAL 60000000000 NANOSECOND = INTERVAL 1 MINUTE; +SELECT INTERVAL 7199999999999 NANOSECOND < INTERVAL 2 HOUR; +SELECT INTERVAL 1 NANOSECOND < INTERVAL 2 DAY; +SELECT INTERVAL 5 NANOSECOND < INTERVAL 1 WEEK; -SELECT toIntervalNanosecond(500) < toIntervalNanosecond(300); -SELECT toIntervalNanosecond(1000) > toIntervalNanosecond(1500); -SELECT toIntervalNanosecond(2000) != toIntervalNanosecond(2000); -SELECT toIntervalNanosecond(1000) < toIntervalMicrosecond(1); -SELECT toIntervalNanosecond(1000001) < toIntervalMillisecond(1); -SELECT toIntervalNanosecond(2000000001) < toIntervalSecond(2); -SELECT toIntervalNanosecond(60000000000) != toIntervalMinute(1); -SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); -SELECT toIntervalNanosecond(1) > toIntervalDay(2); -SELECT toIntervalNanosecond(5) > toIntervalWeek(1); +SELECT INTERVAL 500 NANOSECOND < INTERVAL 300 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND > INTERVAL 1500 NANOSECOND; +SELECT INTERVAL 2000 NANOSECOND != INTERVAL 2000 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND < INTERVAL 1 MICROSECOND; +SELECT INTERVAL 1000001 NANOSECOND < INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000001 NANOSECOND < INTERVAL 2 SECOND; +SELECT INTERVAL 60000000000 NANOSECOND != INTERVAL 1 MINUTE; +SELECT INTERVAL 7199999999999 NANOSECOND > INTERVAL 2 HOUR; +SELECT INTERVAL 1 NANOSECOND > INTERVAL 2 DAY; +SELECT INTERVAL 5 NANOSECOND > INTERVAL 1 WEEK; -SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 1 NANOSECOND < INTERVAL 2 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing microseconds'); -SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); -SELECT toIntervalMicrosecond(1001) > toIntervalMillisecond(1); -SELECT toIntervalMicrosecond(2000000) = toIntervalSecond(2); -SELECT toIntervalMicrosecond(179999999) < toIntervalMinute(3); -SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); -SELECT toIntervalMicrosecond(36000000000000) > toIntervalDay(2); -SELECT toIntervalMicrosecond(1209600000000) = toIntervalWeek(2); +SELECT INTERVAL 1 MICROSECOND < INTERVAL 999 MICROSECOND; +SELECT INTERVAL 1001 MICROSECOND > INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000 MICROSECOND = INTERVAL 2 SECOND; +SELECT INTERVAL 179999999 MICROSECOND < INTERVAL 3 MINUTE; +SELECT INTERVAL 3600000000 MICROSECOND = INTERVAL 1 HOUR; +SELECT INTERVAL 36000000000000 MICROSECOND > INTERVAL 2 DAY; +SELECT INTERVAL 1209600000000 MICROSECOND = INTERVAL 2 WEEK; -SELECT toIntervalMicrosecond(1) > toIntervalMicrosecond(999); -SELECT toIntervalMicrosecond(1001) < toIntervalMillisecond(1); -SELECT toIntervalMicrosecond(2000000) != toIntervalSecond(2); -SELECT toIntervalMicrosecond(179999999) > toIntervalMinute(3); -SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); -SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); -SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); +SELECT INTERVAL 1 MICROSECOND > INTERVAL 999 MICROSECOND; +SELECT INTERVAL 1001 MICROSECOND < INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000 MICROSECOND != INTERVAL 2 SECOND; +SELECT INTERVAL 179999999 MICROSECOND > INTERVAL 3 MINUTE; +SELECT INTERVAL 3600000000 MICROSECOND != INTERVAL 1 HOUR; +SELECT INTERVAL 36000000000000 MICROSECOND < INTERVAL 2 DAY; +SELECT INTERVAL 1209600000000 MICROSECOND != INTERVAL 2 WEEK; -SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 MICROSECOND < INTERVAL 1 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing milliseconds'); -SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); -SELECT toIntervalMillisecond(2000) = toIntervalSecond(2); -SELECT toIntervalMillisecond(170000) < toIntervalMinute(3); -SELECT toIntervalMillisecond(144000001) > toIntervalHour(40); -SELECT toIntervalMillisecond(1728000000) = toIntervalDay(20); -SELECT toIntervalMillisecond(1198599999) < toIntervalWeek(2); +SELECT INTERVAL 2000 MILLISECOND > INTERVAL 2 MILLISECOND; +SELECT INTERVAL 2000 MILLISECOND = INTERVAL 2 SECOND; +SELECT INTERVAL 170000 MILLISECOND < INTERVAL 3 MINUTE; +SELECT INTERVAL 144000001 MILLISECOND > INTERVAL 40 HOUR; +SELECT INTERVAL 1728000000 MILLISECOND = INTERVAL 20 DAY; +SELECT INTERVAL 1198599999 MILLISECOND < INTERVAL 2 WEEK; -SELECT toIntervalMillisecond(2000) < toIntervalMillisecond(2); -SELECT toIntervalMillisecond(2000) != toIntervalSecond(2); -SELECT toIntervalMillisecond(170000) > toIntervalMinute(3); -SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); -SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); -SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); +SELECT INTERVAL 2000 MILLISECOND < INTERVAL 2 MILLISECOND; +SELECT INTERVAL 2000 MILLISECOND != INTERVAL 2 SECOND; +SELECT INTERVAL 170000 MILLISECOND > INTERVAL 3 MINUTE; +SELECT INTERVAL 144000001 MILLISECOND < INTERVAL 40 HOUR; +SELECT INTERVAL 1728000000 MILLISECOND != INTERVAL 20 DAY; +SELECT INTERVAL 1198599999 MILLISECOND > INTERVAL 2 WEEK; -SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 MILLISECOND < INTERVAL 1 YEAR; -- { serverError NO_COMMON_TYPE } SELECT('Comparing seconds'); -SELECT toIntervalSecond(120) > toIntervalSecond(2); -SELECT toIntervalSecond(120) = toIntervalMinute(2); -SELECT toIntervalSecond(1) < toIntervalHour(2); -SELECT toIntervalSecond(86401) >= toIntervalDay(1); -SELECT toIntervalSecond(1209600) = toIntervalWeek(2); +SELECT INTERVAL 120 SECOND > INTERVAL 2 SECOND; +SELECT INTERVAL 120 SECOND = INTERVAL 2 MINUTE; +SELECT INTERVAL 1 SECOND < INTERVAL 2 HOUR; +SELECT INTERVAL 86401 SECOND >= INTERVAL 1 DAY; +SELECT INTERVAL 1209600 SECOND = INTERVAL 2 WEEK; -SELECT toIntervalSecond(120) < toIntervalSecond(2); -SELECT toIntervalSecond(120) != toIntervalMinute(2); -SELECT toIntervalSecond(1) > toIntervalHour(2); -SELECT toIntervalSecond(86401) < toIntervalDay(1); -SELECT toIntervalSecond(1209600) != toIntervalWeek(2); +SELECT INTERVAL 120 SECOND < INTERVAL 2 SECOND; +SELECT INTERVAL 120 SECOND != INTERVAL 2 MINUTE; +SELECT INTERVAL 1 SECOND > INTERVAL 2 HOUR; +SELECT INTERVAL 86401 SECOND < INTERVAL 1 DAY; +SELECT INTERVAL 1209600 SECOND != INTERVAL 2 WEEK; -SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 SECOND < INTERVAL 1 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing minutes'); -SELECT toIntervalMinute(1) < toIntervalMinute(59); -SELECT toIntervalMinute(1) < toIntervalHour(59); -SELECT toIntervalMinute(1440) = toIntervalDay(1); -SELECT toIntervalMinute(30241) > toIntervalWeek(3); +SELECT INTERVAL 1 MINUTE < INTERVAL 59 MINUTE; +SELECT INTERVAL 1 MINUTE < INTERVAL 59 HOUR; +SELECT INTERVAL 1440 MINUTE = INTERVAL 1 DAY; +SELECT INTERVAL 30241 MINUTE > INTERVAL 3 WEEK; -SELECT toIntervalMinute(1) > toIntervalMinute(59); -SELECT toIntervalMinute(1) > toIntervalHour(59); -SELECT toIntervalMinute(1440) != toIntervalDay(1); -SELECT toIntervalMinute(30241) < toIntervalWeek(3); +SELECT INTERVAL 1 MINUTE > INTERVAL 59 MINUTE; +SELECT INTERVAL 1 MINUTE > INTERVAL 59 HOUR; +SELECT INTERVAL 1440 MINUTE != INTERVAL 1 DAY; +SELECT INTERVAL 30241 MINUTE < INTERVAL 3 WEEK; -SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 MINUTE = INTERVAL 120 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing hours'); -SELECT toIntervalHour(48) > toIntervalHour(2); -SELECT toIntervalHour(48) >= toIntervalDay(2); -SELECT toIntervalHour(672) = toIntervalWeek(4); +SELECT INTERVAL 48 HOUR > INTERVAL 2 HOUR; +SELECT INTERVAL 48 HOUR >= INTERVAL 2 DAY; +SELECT INTERVAL 672 HOUR = INTERVAL 4 WEEK; -SELECT toIntervalHour(48) < toIntervalHour(2); -SELECT toIntervalHour(48) < toIntervalDay(2); -SELECT toIntervalHour(672) != toIntervalWeek(4); +SELECT INTERVAL 48 HOUR < INTERVAL 2 HOUR; +SELECT INTERVAL 48 HOUR < INTERVAL 2 DAY; +SELECT INTERVAL 672 HOUR != INTERVAL 4 WEEK; -SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 HOUR < INTERVAL 1 YEAR; -- { serverError NO_COMMON_TYPE } SELECT('Comparing days'); -SELECT toIntervalDay(1) < toIntervalDay(23); -SELECT toIntervalDay(25) > toIntervalWeek(3); +SELECT INTERVAL 1 DAY < INTERVAL 23 DAY; +SELECT INTERVAL 25 DAY > INTERVAL 3 WEEK; -SELECT toIntervalDay(1) > toIntervalDay(23); -SELECT toIntervalDay(25) < toIntervalWeek(3); +SELECT INTERVAL 1 DAY > INTERVAL 23 DAY; +SELECT INTERVAL 25 DAY < INTERVAL 3 WEEK; -SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 DAY = INTERVAL 48 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing weeks'); -SELECT toIntervalWeek(1) < toIntervalWeek(6); +SELECT INTERVAL 1 WEEK < INTERVAL 6 WEEK; -SELECT toIntervalWeek(1) > toIntervalWeek(6); +SELECT INTERVAL 1 WEEK > INTERVAL 6 WEEK; -SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 124 WEEK > INTERVAL 8 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing months'); -SELECT toIntervalMonth(1) < toIntervalMonth(3); -SELECT toIntervalMonth(124) > toIntervalQuarter(5); -SELECT toIntervalMonth(36) = toIntervalYear(3); +SELECT INTERVAL 1 MONTH < INTERVAL 3 MONTH; +SELECT INTERVAL 124 MONTH > INTERVAL 5 QUARTER; +SELECT INTERVAL 36 MONTH = INTERVAL 3 YEAR; -SELECT toIntervalMonth(1) > toIntervalMonth(3); -SELECT toIntervalMonth(124) < toIntervalQuarter(5); -SELECT toIntervalMonth(36) != toIntervalYear(3); +SELECT INTERVAL 1 MONTH > INTERVAL 3 MONTH; +SELECT INTERVAL 124 MONTH < INTERVAL 5 QUARTER; +SELECT INTERVAL 36 MONTH != INTERVAL 3 YEAR; -SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 6 MONTH = INTERVAL 26 MICROSECOND; -- { serverError NO_COMMON_TYPE } SELECT('Comparing quarters'); -SELECT toIntervalQuarter(5) > toIntervalQuarter(4); -SELECT toIntervalQuarter(20) = toIntervalYear(5); +SELECT INTERVAL 5 QUARTER > INTERVAL 4 QUARTER; +SELECT INTERVAL 20 QUARTER = INTERVAL 5 YEAR; -SELECT toIntervalQuarter(5) < toIntervalQuarter(4); -SELECT toIntervalQuarter(20) != toIntervalYear(5); +SELECT INTERVAL 5 QUARTER < INTERVAL 4 QUARTER; +SELECT INTERVAL 20 QUARTER != INTERVAL 5 YEAR; -SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 QUARTER = INTERVAL 6 NANOSECOND; -- { serverError NO_COMMON_TYPE } SELECT('Comparing years'); -SELECT toIntervalYear(1) < toIntervalYear(3); +SELECT INTERVAL 1 YEAR < INTERVAL 3 YEAR; -SELECT toIntervalYear(1) > toIntervalYear(3); +SELECT INTERVAL 1 YEAR > INTERVAL 3 YEAR; -SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 YEAR = INTERVAL 8 SECOND; -- { serverError NO_COMMON_TYPE } \ No newline at end of file From 9c25314a58721c18c265e46276199308ddc6cce0 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 15:28:31 +0000 Subject: [PATCH 096/409] Fixed settings --- src/Core/SettingsChangesHistory.cpp | 12 +++++++++++- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 17 ++++++++++++----- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8f73e10c44f..86988d697b7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -75,7 +75,17 @@ static std::initializer_listgetOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet( + cache_key, file_offset_of_buffer_end, size, file_size.value(), + create_settings, settings.filesystem_cache_segments_batch_size, user); } + return !file_segments->empty(); } @@ -158,8 +161,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current offset: {}", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + "Having {} file segments to read: {}, current read range: [{}, {})", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); initialized = true; } @@ -1043,6 +1046,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; + + LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", + remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); + size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1055,8 +1062,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {})", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); + fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); } swap(*implementation_buffer); From 25ce6df8957c36358498aed83ce2e17d034261f3 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 16:22:30 +0000 Subject: [PATCH 097/409] Return local table --- .../TableFunctionObjectStorage.cpp | 2 +- .../TableFunctionObjectStorage.h | 22 +++++++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..6235108e2ba 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -222,5 +222,5 @@ template class TableFunctionObjectStorage template class TableFunctionObjectStorage; template class TableFunctionObjectStorage; #endif - +template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From 4c043301e6dde6b0c83394d6721e112c9c7bf4ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Aug 2024 10:30:31 +0200 Subject: [PATCH 098/409] Avoid ignoring errors of execute_process() (set COMMAND_ERROR_IS_FATAL=ANY) This will fix with issues like this [1]: Aug 12 09:58:44 '/usr/bin/cmake' '--build' '/build/build_docker/native' '--target' 'pre_compressor' Aug 12 09:58:44 sccache: error: Server startup failed: cache storage failed to read: Unexpected (temporary) at stat Aug 12 09:58:45 ninja: build stopped: subcommand failed. Aug 12 09:58:45 -- Configuring done (77.7s) Aug 12 09:58:47 -- Generating done (1.8s) Aug 12 09:58:47 -- Build files have been written to: /build/build_docker So as you can see even if ninja fails it still wrote build files, while it should fail. [1]: https://s3.amazonaws.com/clickhouse-test-reports/64955/0af41e32a5822d25ac3760f1ebb2313557474701/builds/report.html [2]: https://s3.amazonaws.com/clickhouse-builds/PRs/64955/0af41e32a5822d25ac3760f1ebb2313557474701/binary_darwin_aarch64/build_log.log Note, COMMAND_ERROR_IS_FATAL is 3.19+, and the requirement for now is 3.20 Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 12 ++++++-- PreLoad.cmake | 10 +++++-- cmake/freebsd/default_libs.cmake | 12 ++++++-- cmake/linux/default_libs.cmake | 6 +++- cmake/tools.cmake | 6 +++- cmake/utils.cmake | 5 +++- contrib/cctz-cmake/CMakeLists.txt | 4 ++- contrib/google-protobuf-cmake/CMakeLists.txt | 12 ++++++-- contrib/grpc-cmake/CMakeLists.txt | 30 +++++++++++++------ .../completions/CMakeLists.txt | 1 + 10 files changed, 75 insertions(+), 23 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7b4e0484ab1..8e2302e6c52 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -609,7 +609,9 @@ if (NATIVE_BUILD_TARGETS execute_process( COMMAND ${CMAKE_COMMAND} -E make_directory "${NATIVE_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} @@ -621,9 +623,13 @@ if (NATIVE_BUILD_TARGETS "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=${ENABLE_CLICKHOUSE_SELF_EXTRACTING}" ${PROJECT_SOURCE_DIR} WORKING_DIRECTORY "${NATIVE_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${NATIVE_BUILD_DIR}" --target ${NATIVE_BUILD_TARGETS} - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) endif () diff --git a/PreLoad.cmake b/PreLoad.cmake index e0fd37b2fd6..92b221c9f63 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -51,8 +51,14 @@ if (NOT "$ENV{CFLAGS}" STREQUAL "" endif() # Default toolchain - this is needed to avoid dependency on OS files. -execute_process(COMMAND uname -s OUTPUT_VARIABLE OS) -execute_process(COMMAND uname -m OUTPUT_VARIABLE ARCH) +execute_process(COMMAND uname -s + OUTPUT_VARIABLE OS + COMMAND_ERROR_IS_FATAL ANY +) +execute_process(COMMAND uname -m + OUTPUT_VARIABLE ARCH + COMMAND_ERROR_IS_FATAL ANY +) # By default, prefer clang on Linux # But note, that you still may change the compiler with -DCMAKE_C_COMPILER/-DCMAKE_CXX_COMPILER. diff --git a/cmake/freebsd/default_libs.cmake b/cmake/freebsd/default_libs.cmake index 6bde75f8c9a..3f5b3829877 100644 --- a/cmake/freebsd/default_libs.cmake +++ b/cmake/freebsd/default_libs.cmake @@ -9,10 +9,18 @@ endif () file(GLOB bprefix "/usr/local/llvm${COMPILER_VERSION_MAJOR}/lib/clang/${COMPILER_VERSION_MAJOR}/lib/${system_processor}-portbld-freebsd*/") message(STATUS "-Bprefix: ${bprefix}") -execute_process(COMMAND ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins-${system_processor}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process(COMMAND + ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins-${system_processor}.a + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) # --print-file-name simply prints what you passed in case of nothing was resolved, so let's try one other possible option if (BUILTINS_LIBRARY STREQUAL "libclang_rt.builtins-${system_processor}.a") - execute_process(COMMAND ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) + execute_process(COMMAND + ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins.a + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) endif() if (BUILTINS_LIBRARY STREQUAL "libclang_rt.builtins.a") message(FATAL_ERROR "libclang_rt.builtins had not been found") diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 4a06243243e..51620bc9f33 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -5,7 +5,11 @@ set (DEFAULT_LIBS "-nodefaultlibs") # We need builtins from Clang's RT even without libcxx - for ubsan+int128. # See https://bugs.llvm.org/show_bug.cgi?id=16404 -execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process (COMMAND + ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) # Apparently, in clang-19, the UBSan support library for C++ was moved out into ubsan_standalone_cxx.a, so we have to include both. if (SANITIZE STREQUAL undefined) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 7aa5d4c51ce..5c7da54b779 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -5,7 +5,11 @@ if (NOT CMAKE_CXX_COMPILER_ID MATCHES "Clang") endif () # Print details to output -execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version OUTPUT_VARIABLE COMPILER_SELF_IDENTIFICATION OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version + OUTPUT_VARIABLE COMPILER_SELF_IDENTIFICATION + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE +) message (STATUS "Using compiler:\n${COMPILER_SELF_IDENTIFICATION}") # Require minimum compiler versions diff --git a/cmake/utils.cmake b/cmake/utils.cmake index a318408098a..a99d8e050a8 100644 --- a/cmake/utils.cmake +++ b/cmake/utils.cmake @@ -90,7 +90,10 @@ endfunction() # Function get_cmake_properties returns list of all propreties that cmake supports function(get_cmake_properties outvar) - execute_process(COMMAND cmake --help-property-list OUTPUT_VARIABLE cmake_properties) + execute_process(COMMAND cmake --help-property-list + OUTPUT_VARIABLE cmake_properties + COMMAND_ERROR_IS_FATAL ANY + ) # Convert command output into a CMake list string(REGEX REPLACE ";" "\\\\;" cmake_properties "${cmake_properties}") string(REGEX REPLACE "\n" ";" cmake_properties "${cmake_properties}") diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7161f743de1..fadf948b053 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -37,7 +37,9 @@ message(STATUS "Packaging with tzdata version: ${TZDATA_VERSION}") execute_process(COMMAND bash -c "cd ${TZDIR} && find * -type f -and ! -name '*.tab' -and ! -name 'localtime' | LC_ALL=C sort | paste -sd ';' -" OUTPUT_STRIP_TRAILING_WHITESPACE - OUTPUT_VARIABLE TIMEZONES) + OUTPUT_VARIABLE TIMEZONES + COMMAND_ERROR_IS_FATAL ANY +) file(APPEND ${TIMEZONES_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") file(APPEND ${TIMEZONES_FILE} "#include \n") diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index e44f737cfc3..f1a744f851f 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -359,7 +359,9 @@ else () execute_process( COMMAND mkdir -p ${PROTOC_BUILD_DIR} - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} @@ -375,11 +377,15 @@ else () "-DABSL_ENABLE_INSTALL=0" "${protobuf_source_dir}" WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) endif () add_executable(protoc IMPORTED GLOBAL) diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 1c0bf41ff78..975774d1990 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -51,8 +51,9 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME set(OPENSSL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake") execute_process( - COMMAND mkdir -p ${OPENSSL_BUILD_DIR} - COMMAND_ECHO STDOUT + COMMAND mkdir -p ${OPENSSL_BUILD_DIR} + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY ) if (CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") @@ -89,15 +90,21 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME "-DClickHouse_SOURCE_DIR=${ClickHouse_SOURCE_DIR}" "${OPENSSL_SOURCE_DIR}" WORKING_DIRECTORY "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --install "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) # It's not important on which file we depend, we just want to specify right order add_library(openssl_for_grpc STATIC IMPORTED GLOBAL) @@ -108,8 +115,9 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME set (GRPC_CPP_PLUGIN_BUILD_DIR "${_gRPC_BINARY_DIR}/build") execute_process( - COMMAND mkdir -p ${GRPC_CPP_PLUGIN_BUILD_DIR} - COMMAND_ECHO STDOUT + COMMAND mkdir -p ${GRPC_CPP_PLUGIN_BUILD_DIR} + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY ) set(abseil_source_dir "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") @@ -140,11 +148,15 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME "-DgRPC_SSL_PROVIDER=package" "${_gRPC_SOURCE_DIR}" WORKING_DIRECTORY "${GRPC_CPP_PLUGIN_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${GRPC_CPP_PLUGIN_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) add_executable(grpc_cpp_plugin IMPORTED GLOBAL) set_target_properties (grpc_cpp_plugin PROPERTIES IMPORTED_LOCATION "${GRPC_CPP_PLUGIN_BUILD_DIR}/grpc_cpp_plugin") diff --git a/programs/bash-completion/completions/CMakeLists.txt b/programs/bash-completion/completions/CMakeLists.txt index d364e07ef6e..2e911e81981 100644 --- a/programs/bash-completion/completions/CMakeLists.txt +++ b/programs/bash-completion/completions/CMakeLists.txt @@ -6,6 +6,7 @@ macro(configure_bash_completion) COMMAND ${PKG_CONFIG_BIN} --variable=completionsdir bash-completion OUTPUT_VARIABLE ${out} OUTPUT_STRIP_TRAILING_WHITESPACE + COMMAND_ERROR_IS_FATAL ANY ) endif() string(REPLACE /usr "${CMAKE_INSTALL_PREFIX}" out "${out}") From 7f005a6ca48d4f193470d3a71bc1d97ff55f4a2f Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Tue, 13 Aug 2024 08:38:30 +0000 Subject: [PATCH 099/409] Fix small value DateTime64 constant folding in nested query --- src/Analyzer/ConstantNode.cpp | 10 ++++- ...222_datetime64_small_value_const.reference | 18 +++++++++ .../03222_datetime64_small_value_const.sql | 39 +++++++++++++++++++ 3 files changed, 65 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03222_datetime64_small_value_const.reference create mode 100644 tests/queries/0_stateless/03222_datetime64_small_value_const.sql diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index c65090f5b55..3d0f448da4b 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -177,9 +177,15 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const * It could also lead to ambiguous parsing because we don't know if the string literal represents a date or a Decimal64 literal. * For this reason, we use a string literal representing a date instead of a Decimal64 literal. */ - if (WhichDataType(constant_value_type->getTypeId()).isDateTime64()) + if ((WhichDataType(constant_value_type->getTypeId()).isDateTime64()) || + (WhichDataType(constant_value_type->getTypeId()).isNullable() && WhichDataType((typeid_cast(constant_value_type.get()))->getNestedType()->getTypeId()).isDateTime64())) { - const auto * date_time_type = typeid_cast(constant_value_type.get()); + const DataTypeDateTime64 * date_time_type = nullptr; + if (WhichDataType(constant_value_type->getTypeId()).isNullable()) + date_time_type = typeid_cast((typeid_cast(constant_value_type.get()))->getNestedType().get()); + else + date_time_type = typeid_cast(constant_value_type.get()); + DecimalField decimal_value; if (constant_value_literal.tryGet>(decimal_value)) { diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.reference b/tests/queries/0_stateless/03222_datetime64_small_value_const.reference new file mode 100644 index 00000000000..ae36c08acc5 --- /dev/null +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.reference @@ -0,0 +1,18 @@ +0 1970-01-01 00:00:00.000 +0 1970-01-01 00:00:05.000 +0 1970-01-01 00:45:25.456789 +0 1970-01-01 00:53:25.456789123 +0 \N +1 1970-01-01 00:00:00.000 +5 1970-01-01 00:00:00.000 +2 1970-01-01 00:00:02.456 +3 1970-01-01 00:00:04.811 +4 1970-01-01 00:10:05.000 +4 1970-01-01 00:10:05.000 +1 1970-01-01 00:00:00.000 +2 1970-01-01 00:00:02.456 +3 1970-01-01 00:00:04.811 +5 1970-01-01 00:00:00.000 +0 +0 +5 diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql new file mode 100644 index 00000000000..6999ba9662a --- /dev/null +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -0,0 +1,39 @@ +-- Tags: shard + +select *, (select toDateTime64(0, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64(5, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64('1970-01-01 00:45:25.456789', 6)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64('1970-01-01 00:53:25.456789123', 9)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64(null,3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; + +create database if not exists shard_0; +create database if not exists shard_1; + +drop table if exists shard_0.dt64_03222; +drop table if exists shard_1.dt64_03222; +drop table if exists distr_03222_dt64; + +create table shard_0.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; +create table shard_1.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; +create table distr_03222_dt64 (id UInt64, dt DateTime64(3)) engine = Distributed(test_cluster_two_shards_different_databases, '', dt64_03222); + +insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)) +insert into shard_0.dt64_03222 values(2, toDateTime64('1970-01-01 00:00:02.456',3)); +insert into shard_1.dt64_03222 values(3, toDateTime64('1970-01-01 00:00:04.811',3)); +insert into shard_1.dt64_03222 values(4, toDateTime64('1970-01-01 00:10:05',3)); +insert into shard_1.dt64_03222 values(5, toDateTime64(0,3)); + +--Output : 1,5 2,3,4 4 1,2,3,5 0 0 5 +select id, dt from distr_03222_dt64 where dt = (select toDateTime64(0,3)) order by id; +select id, dt from distr_03222_dt64 where dt > (select toDateTime64(0,3)) order by id; +select id, dt from distr_03222_dt64 where dt > (select toDateTime64('1970-01-01 00:10:00.000',3)) order by id; +select id, dt from distr_03222_dt64 where dt < (select toDateTime64(5,3)) order by id; + +select count(*) from distr_03222_dt64 where dt > (select toDateTime64('2024-07-20 00:00:00',3)); +select count(*) from distr_03222_dt64 where dt > (select now()); +select count(*) from distr_03222_dt64 where dt < (select toDateTime64('2004-07-20 00:00:00',3)); + + +drop table if exists shard_0.dt64_03222; +drop table if exists shard_1.dt64_03222; +drop table if exists distr_03222_dt64; From e46c5a75ef6b5488834add56bea4cab327515bfb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 13 Aug 2024 17:59:05 +0800 Subject: [PATCH 100/409] fix building issue --- src/Functions/{FunctionOverlay.cpp => overlay.cpp} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename src/Functions/{FunctionOverlay.cpp => overlay.cpp} (99%) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/overlay.cpp similarity index 99% rename from src/Functions/FunctionOverlay.cpp rename to src/Functions/overlay.cpp index 61d2df88ab1..094da27a71d 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/overlay.cpp @@ -728,8 +728,8 @@ private: REGISTER_FUNCTION(Overlay) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Sensitive); } } From 534ec1fa01b9de4206a8d6e3f91d73d7d1b2f743 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 10:12:45 +0000 Subject: [PATCH 101/409] Correct compile error --- src/TableFunctions/TableFunctionObjectStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 6235108e2ba..0ddf901ee21 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -14,10 +14,11 @@ #include #include -#include -#include -#include #include +#include +#include +#include +#include namespace DB From 0414cdbbbf32efe10a92c9dd93ba47743ceeb848 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 15:58:49 +0200 Subject: [PATCH 102/409] Fix unpack error --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5946e561949..5bde4686d3a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1995,7 +1995,7 @@ class TestSuite: tag_line = find_tag_line(file) next_line = file.readline() except UnicodeDecodeError: - return [] + return [], {} try: if filepath.endswith(".sql"): for line in file: From be92986eda3298f7f44d64509bda83689713a2a0 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 14:48:29 +0000 Subject: [PATCH 103/409] Possibly fix broken tests and build --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h | 2 +- src/TableFunctions/registerDataLakeTableFunctions.cpp | 2 -- tests/integration/helpers/s3_tools.py | 2 +- 6 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..949f994fa09 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -3,7 +3,7 @@ #include "config.h" #include -#if USE_AWS_S3 && USE_PARQUET +#if USE_PARQUET #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..ae0e3066ffb 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -2,7 +2,7 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 6d18b13df01..9b9d92e282c 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -1,6 +1,6 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index 9476ac6e7d9..7b0deab91c3 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -1,6 +1,6 @@ #pragma once -#if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. #include #include diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 8de852f4fba..8361d8a7977 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -75,10 +75,8 @@ void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AVRO -# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#endif #if USE_AWS_S3 # if USE_PARQUET registerTableFunctionDeltaLake(factory); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/s3_tools.py index 39c56e0eb85..5b727060e69 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/s3_tools.py @@ -85,7 +85,7 @@ class AzureUploader(CloudUploader): def upload_directory(minio_client, bucket, local_path, remote_path): - S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + return S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( local_path, remote_path ) From 6dfed409f460311f133e30e70f839f9865d71861 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 16:09:45 +0000 Subject: [PATCH 104/409] Fix seraching for query params --- tests/clickhouse-test | 35 ++++++++++++++++++++++------------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5bde4686d3a..515b519af3e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1978,13 +1978,20 @@ class TestSuite: def is_shebang(line: str) -> bool: return line.startswith("#!") - def find_tag_line(file): - line = file.readline() - while line != "": - line = line.strip() - if line and not is_shebang(line): + def find_tag_line(lines, comment_sign): + for line in lines: + if line.startswith(comment_sign) and line[ + len(comment_sign) : + ].lstrip().startswith("Tags:"): + return line + return "" + + def find_random_settings_limits_line(lines, comment_sign): + for line in lines: + if line.startswith(comment_sign) and line[ + len(comment_sign) : + ].lstrip().startswith("Random settings limits:"): return line - line = file.readline() return "" def load_tags_and_random_settings_limits_from_file(filepath): @@ -1992,13 +1999,16 @@ class TestSuite: need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: - tag_line = find_tag_line(file) - next_line = file.readline() + lines = file.readlines() + tag_line = find_tag_line(lines, comment_sign) + random_settings_limits_line = find_random_settings_limits_line( + lines, comment_sign + ) except UnicodeDecodeError: return [], {} try: if filepath.endswith(".sql"): - for line in file: + for line in lines: if "{CLICKHOUSE_DATABASE" in line: need_query_params = True except UnicodeDecodeError: @@ -2006,7 +2016,6 @@ class TestSuite: parsed_tags = parse_tags_from_line(tag_line, comment_sign) if need_query_params: parsed_tags.add("need-query-parameters") - random_settings_limits_line = next_line if parsed_tags else tag_line random_settings_limits = parse_random_settings_limits_from_line( random_settings_limits_line, comment_sign ) @@ -2068,9 +2077,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( - all_tags_and_random_settings_limits[1] - ) + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 3cfb921befa895e445e8d7b98e639015e1e41aa0 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 18:41:53 +0000 Subject: [PATCH 105/409] 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 106/409] 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 0abca8b7ddbafa37da5b1196b21fb816999fd334 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 14 Aug 2024 09:57:59 +0800 Subject: [PATCH 107/409] fix doc --- .../sql-reference/functions/string-replace-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 1caa6215b6b..d086c9ee64b 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -263,8 +263,8 @@ overlay(s, replace, position[, length]) - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int.md). -- `length`: Optional. An integer type [Int](../data-types/int.md). +- `position`: An integer type [Int](../data-types/int-uint.md). +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** @@ -300,8 +300,8 @@ overlayUTF8(s, replace, position[, length]) - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int.md). -- `length`: Optional. An integer type [Int](../data-types/int.md). +- `position`: An integer type [Int](../data-types/int-uint.md). +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** From 6bed26a52764558ba2b52752bdfd126fd943c616 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 14 Aug 2024 02:19:34 +0000 Subject: [PATCH 108/409] 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 f740cf4eaa71621fb518c6d5668e8356f452a979 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Aug 2024 09:54:03 +0200 Subject: [PATCH 109/409] Fix data race on SampleKey --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a6ef0063069..6efd3a5c97f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -369,7 +369,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// If sample and final are used together no need to calculate sampling expression twice. /// The first time it was calculated for final, because sample key is a part of the PK. /// So, assume that we already have calculated column. - ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); + ASTPtr sampling_key_ast; if (final) { @@ -377,6 +377,12 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// We do spoil available_real_columns here, but it is not used later. available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); } + else + { + sampling_key_ast = metadata_snapshot->getSamplingKeyAST()->clone(); + } + + chassert(sampling_key_ast != nullptr); if (has_lower_limit) { From 9754620c077b85f5b2cb837416a73f3c831842e2 Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Wed, 14 Aug 2024 17:51:45 +0800 Subject: [PATCH 110/409] Update 02352_lightweight_delete_in_partition.sql update test From 4f77893c79d3c2c8b0304837bd45033301e6d49a Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Wed, 14 Aug 2024 18:24:03 +0800 Subject: [PATCH 111/409] Update ParserDeleteQuery.cpp fix: delete space --- src/Parsers/ParserDeleteQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 3503fbe9ad2..d7cdcd920ad 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -62,7 +62,7 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; - + if (query->partition) query->children.push_back(query->partition); From 476819a56fd71e445da5aa91f34e10abb8ded327 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 14 Aug 2024 11:35:37 +0000 Subject: [PATCH 112/409] Changed test to work in parallel --- .../integration/test_storage_iceberg/test.py | 130 +++++++++++++----- 1 file changed, 97 insertions(+), 33 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b66133f5562..176c7e209bd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -165,9 +165,8 @@ def generate_data(spark, start, end): return df -def create_iceberg_table( +def get_creation_expression( storage_type, - node, table_name, cluster, format="Parquet", @@ -182,38 +181,52 @@ def create_iceberg_table( print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" elif storage_type == "azure": if table_function: return f""" icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" elif storage_type == "local": if table_function: return f""" icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" else: raise Exception(f"Unknown iceberg storage type: {storage_type}") +def get_uuid_str(): + return str(uuid.uuid4()).replace("-", "_") + + +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + **kwargs, +): + node.query( + get_creation_expression(storage_type, table_name, cluster, format, **kwargs) + ) + + def create_initial_data_file( cluster, node, query, table_name, compression_method="none" ): @@ -258,7 +271,14 @@ def default_upload_directory( def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_single_iceberg_file_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -281,7 +301,14 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_partition_by_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -311,7 +338,14 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_multiple_iceberg_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -364,7 +398,9 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_types_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_types_" + format_version + "_" + storage_type + "_" + get_uuid_str() + ) data = [ ( @@ -404,8 +440,8 @@ def test_types(started_cluster, format_version, storage_type): == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function_expr = create_iceberg_table( - storage_type, instance, TABLE_NAME, started_cluster, table_function=True + table_function_expr = get_creation_expression( + storage_type, TABLE_NAME, started_cluster, table_function=True ) assert ( instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() @@ -430,7 +466,14 @@ def test_delete_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_delete_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -496,7 +539,14 @@ def test_evolved_schema(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_evolved_schema_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -542,7 +592,7 @@ def test_row_based_deletes(started_cluster, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes_" + storage_type + TABLE_NAME = "test_row_based_deletes_" + storage_type + "_" + get_uuid_str() spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -587,6 +637,8 @@ def test_schema_inference(started_cluster, format_version, storage_type): + format_version + "_" + storage_type + + "_" + + get_uuid_str() ) # Types time, timestamptz, fixed are not supported in Spark. @@ -645,7 +697,14 @@ def test_schema_inference(started_cluster, format_version, storage_type): def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_metadata_selection_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -674,7 +733,12 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = ( - "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + "test_metadata_selection_with_uuid_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() ) spark.sql( @@ -689,7 +753,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage for i in range(50): os.rename( f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json", - f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", + f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{get_uuid_str()}.metadata.json", ) files = default_upload_directory( @@ -707,7 +771,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage def test_restart_broken_s3(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_restart_broken_table_function_s3" + TABLE_NAME = "test_restart_broken_table_function_s3" + "_" + get_uuid_str() minio_client = started_cluster.minio_client bucket = "broken2" 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 113/409] 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 114/409] 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 28b0aad3f9e54beed27ee384ab81312233abaa84 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:16:34 +0200 Subject: [PATCH 115/409] Fix python style --- tests/clickhouse-test | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 515b519af3e..8c2da7334d4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1975,9 +1975,6 @@ class TestSuite: ) return random_settings_limits - def is_shebang(line: str) -> bool: - return line.startswith("#!") - def find_tag_line(lines, comment_sign): for line in lines: if line.startswith(comment_sign) and line[ @@ -2077,9 +2074,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( + all_tags_and_random_settings_limits[1] + ) self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 844cdd8937cce17060ea8b54fdfc2428d3015f44 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 14 Aug 2024 20:38:09 +0200 Subject: [PATCH 116/409] update toInterval functions --- .../functions/type-conversion-functions.md | 400 +++++++++++++++++- 1 file changed, 380 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1e618b8cdab..cd6fd9ab839 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4866,30 +4866,23 @@ Result: └───────┴───────────────┴──────┴──────────────┴──────────────┴──────────────────────┘ ``` -## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second) +## toIntervalYear -Converts a Number type argument to an [Interval](../data-types/special-data-types/interval.md) data type. +Returns an interval of `n` years of data type [IntervalYear](../data-types/special-data-types/interval.md). **Syntax** ``` sql -toIntervalSecond(number) -toIntervalMinute(number) -toIntervalHour(number) -toIntervalDay(number) -toIntervalWeek(number) -toIntervalMonth(number) -toIntervalQuarter(number) -toIntervalYear(number) +toIntervalYear(n) ``` **Arguments** -- `number` — Duration of interval. Positive integer number. +- `n` — Number of years. Positive integer number. [Int*](../data-types/int-uint.md). **Returned values** -- The value in `Interval` data type. +- Interval of `n` years. [IntervalYear](../data-types/special-data-types/interval.md). **Example** @@ -4898,19 +4891,386 @@ Query: ``` sql WITH toDate('2019-01-01') AS date, - INTERVAL 1 WEEK AS interval_week, - toIntervalWeek(1) AS interval_to_week -SELECT - date + interval_week, - date + interval_to_week; + toIntervalYear(1) AS interval_to_year +SELECT date + interval_to_year ``` Result: ```response -┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ -│ 2019-01-08 │ 2019-01-08 │ -└───────────────────────────┴──────────────────────────────┘ +┌─plus(date, interval_to_year)─┐ +│ 2020-01-01 │ +└──────────────────────────────┘ +``` + +## toIntervalQuarter + +Returns an interval of `n` quarters of data type [IntervalQuarter](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalQuarter(n) +``` + +**Arguments** + +- `n` — Number of quarters. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` quarters. [IntervalQuarter](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalQuarter(1) AS interval_to_quarter +SELECT date + interval_to_quarter +``` + +Result: + +```response +┌─plus(date, interval_to_quarter)─┐ +│ 2019-04-01 │ +└─────────────────────────────────┘ +``` + +## toIntervalMonth + +Returns an interval of `n` months of data type [IntervalMonth](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMonth(n) +``` + +**Arguments** + +- `n` — Number of m. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` months. [IntervalMonth](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalMonth(1) AS interval_to_month +SELECT date + interval_to_month +``` + +Result: + +```response +┌─plus(date, interval_to_month)─┐ +│ 2019-02-01 │ +└───────────────────────────────┘ +``` + +## toIntervalWeek + +Returns an interval of `n` weeks of data type [IntervalWeek](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalWeek(n) +``` + +**Arguments** + +- `n` — Number of weeks. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` weeks. [IntervalWeek](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalWeek(1) AS interval_to_week +SELECT date + interval_to_week +``` + +Result: + +```response +┌─plus(date, interval_to_week)─┐ +│ 2019-01-08 │ +└──────────────────────────────┘ +``` + +## toIntervalDay + +Returns an interval of `n` days of data type [IntervalDay](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalDay(n) +``` + +**Arguments** + +- `n` — Number of days. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` days. [IntervalDay](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalDay(5) AS interval_to_days +SELECT date + interval_to_days +``` + +Result: + +```response +┌─plus(date, interval_to_days)─┐ +│ 2019-01-06 │ +└──────────────────────────────┘ +``` + +## toIntervalHour + +Returns an interval of `n` hours of data type [IntervalHour](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalHour(n) +``` + +**Arguments** + +- `n` — Number of hours. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` hours. [IntervalHour](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalHour(12) AS interval_to_hours +SELECT date + interval_to_hours +``` + +Result: + +```response +┌─plus(date, interval_to_hours)─┐ +│ 2019-01-01 12:00:00 │ +└───────────────────────────────┘ +``` + +## toIntervalMinute + +Returns an interval of `n` minutes of data type [IntervalMinute](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMinute(n) +``` + +**Arguments** + +- `n` — Number of minutes. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` minutes. [IntervalMinute](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalMinute(12) AS interval_to_minutes +SELECT date + interval_to_minutes +``` + +Result: + +```response +┌─plus(date, interval_to_minutes)─┐ +│ 2019-01-01 00:12:00 │ +└─────────────────────────────────┘ +``` + +## toIntervalSecond + +Returns an interval of `n` seconds of data type [IntervalSecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalSecond(n) +``` + +**Arguments** + +- `n` — Number of seconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` seconds. [IntervalSecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalSecond(30) AS interval_to_seconds +SELECT date + interval_to_seconds +``` + +Result: + +```response +┌─plus(date, interval_to_seconds)─┐ +│ 2019-01-01 00:00:30 │ +└─────────────────────────────────┘ +``` + +## toIntervalMillisecond + +Returns an interval of `n` milliseconds of data type [IntervalMillisecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMillisecond(n) +``` + +**Arguments** + +- `n` — Number of milliseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` milliseconds. [IntervalMilliseconds](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalMillisecond(30) AS interval_to_milliseconds +SELECT date + interval_to_milliseconds +``` + +Result: + +```response +┌─plus(date, interval_to_milliseconds)─┐ +│ 2019-01-01 00:00:00.030 │ +└──────────────────────────────────────┘ +``` + +## toIntervalMicrosecond + +Returns an interval of `n` microseconds of data type [IntervalMicrosecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMicrosecond(n) +``` + +**Arguments** + +- `n` — Number of microseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` microseconds. [IntervalMicrosecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalMicrosecond(30) AS interval_to_microseconds +SELECT date + interval_to_microseconds +``` + +Result: + +```response +┌─plus(date, interval_to_microseconds)─┐ +│ 2019-01-01 00:00:00.000030 │ +└──────────────────────────────────────┘ +``` + +## toIntervalNanosecond + +Returns an interval of `n` nanoseconds of data type [IntervalNanosecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalNanosecond(n) +``` + +**Arguments** + +- `n` — Number of nanoseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` nanoseconds. [IntervalNanosecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalNanosecond(30) AS interval_to_nanoseconds +SELECT date + interval_to_nanoseconds +``` + +Result: + +```response +┌─plus(date, interval_to_nanoseconds)─┐ +│ 2019-01-01 00:00:00.000000030 │ +└─────────────────────────────────────┘ ``` ## parseDateTime From d88299b94dc9563f494d6f1ad9374d1055ad13a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 14 Aug 2024 23:46:40 +0200 Subject: [PATCH 117/409] check that merge entries are valid --- src/Storages/MergeTree/MergeList.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 8fbb163384e..2d48ed4537a 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context) : table_id{table_id_} , partition_id{future_part->part_info.partition_id} @@ -21,8 +26,15 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} { + auto format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; + if (result_part_name != result_part_info.getPartNameV1()) + format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; + for (const auto & source_part : future_part->parts) { + if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); + source_part_names.emplace_back(source_part->name); source_part_paths.emplace_back(source_part->getDataPartStorage().getFullPath()); @@ -42,6 +54,9 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta part->partition.serializeText(part->storage, out, {}); } + if (is_mutation && future_part->parts.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}", future_part->parts.size(), result_part_info.getPartNameV1()); + thread_group = ThreadGroup::createForBackgroundProcess(context); } From 88fa8cb5bb52c7efd2b8e132236ee10ba0dd33d4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 15 Aug 2024 00:06:41 +0200 Subject: [PATCH 118/409] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 2d48ed4537a..1eb2e707194 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -32,7 +32,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta for (const auto & source_part : future_part->parts) { - if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) + if (!source_part->getParentPart() && !result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); source_part_names.emplace_back(source_part->name); From b82c231886f2496c01b288a138663c4d430fc7b2 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Thu, 15 Aug 2024 10:37:13 +0000 Subject: [PATCH 119/409] Code review feedback - used removeNullable() --- src/Analyzer/ConstantNode.cpp | 11 +++-------- .../03222_datetime64_small_value_const.sql | 2 +- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 3d0f448da4b..3a99ad08ad8 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -177,15 +177,10 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const * It could also lead to ambiguous parsing because we don't know if the string literal represents a date or a Decimal64 literal. * For this reason, we use a string literal representing a date instead of a Decimal64 literal. */ - if ((WhichDataType(constant_value_type->getTypeId()).isDateTime64()) || - (WhichDataType(constant_value_type->getTypeId()).isNullable() && WhichDataType((typeid_cast(constant_value_type.get()))->getNestedType()->getTypeId()).isDateTime64())) + const auto & constant_value_end_type = removeNullable(constant_value_type); /// if Nullable + if (WhichDataType(constant_value_end_type->getTypeId()).isDateTime64()) { - const DataTypeDateTime64 * date_time_type = nullptr; - if (WhichDataType(constant_value_type->getTypeId()).isNullable()) - date_time_type = typeid_cast((typeid_cast(constant_value_type.get()))->getNestedType().get()); - else - date_time_type = typeid_cast(constant_value_type.get()); - + const auto * date_time_type = typeid_cast(constant_value_end_type.get()); DecimalField decimal_value; if (constant_value_literal.tryGet>(decimal_value)) { diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql index 6999ba9662a..af06a622f8d 100644 --- a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -17,7 +17,7 @@ create table shard_0.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree create table shard_1.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; create table distr_03222_dt64 (id UInt64, dt DateTime64(3)) engine = Distributed(test_cluster_two_shards_different_databases, '', dt64_03222); -insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)) +insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)); insert into shard_0.dt64_03222 values(2, toDateTime64('1970-01-01 00:00:02.456',3)); insert into shard_1.dt64_03222 values(3, toDateTime64('1970-01-01 00:00:04.811',3)); insert into shard_1.dt64_03222 values(4, toDateTime64('1970-01-01 00:10:05',3)); From df07b07cb9c89200f51ffed4fffd17cb2b0307e8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 12:18:02 +0000 Subject: [PATCH 120/409] Add new setting --- contrib/NuRaft | 2 +- contrib/azure | 2 +- contrib/icu | 2 +- contrib/icudata | 2 +- contrib/libprotobuf-mutator | 2 +- contrib/librdkafka | 2 +- contrib/libunwind | 2 +- contrib/numactl | 1 + contrib/qpl | 2 +- contrib/rocksdb | 2 +- contrib/usearch | 2 +- contrib/zlib-ng | 2 +- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 3 ++- src/Storages/ObjectStorage/Local/Configuration.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 2 +- 16 files changed, 17 insertions(+), 14 deletions(-) create mode 160000 contrib/numactl diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce diff --git a/contrib/azure b/contrib/azure index ea3e19a7be0..67272b7ee0a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 +Subproject commit 67272b7ee0adff6b69921b26eb071ba1a353062c diff --git a/contrib/icu b/contrib/icu index a56dde820dc..7750081bda4 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668 +Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 diff --git a/contrib/icudata b/contrib/icudata index c8e717892a5..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit c8e717892a557b4d2852317c7d628aacc0a0e5ab +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..1f95f808306 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d diff --git a/contrib/librdkafka b/contrib/librdkafka index 2d2aab6f5b7..39d4ed49ccf 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 +Subproject commit 39d4ed49ccf3406e2bf825d5d7b0903b5a290782 diff --git a/contrib/libunwind b/contrib/libunwind index 8f28e64d158..a89d904befe 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 +Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 diff --git a/contrib/numactl b/contrib/numactl new file mode 160000 index 00000000000..8d13d63a05f --- /dev/null +++ b/contrib/numactl @@ -0,0 +1 @@ +Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 diff --git a/contrib/qpl b/contrib/qpl index d4715e0e798..c2ced94c53c 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit d4715e0e79896b85612158e135ee1a85f3b3e04d +Subproject commit c2ced94c53c1ee22191201a59878e9280bc9b9b8 diff --git a/contrib/rocksdb b/contrib/rocksdb index be366233921..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit be366233921293bd07a84dc4ea6991858665f202 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/usearch b/contrib/usearch index 955c6f9c11a..30810452bec 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f +Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356 diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 50f0eae1a41..a2fbeffdc30 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 50f0eae1a411764cd6d1e85b3ce471438acd3c1c +Subproject commit a2fbeffdc30a8b0ce6d54ee31208e2688eac4c9f diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b19ef1000c1..acc02cc4b92 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -113,6 +113,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ + M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7c163750a80..7d1aab939f8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,7 +81,8 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.engine_file_truncate_on_insert, + .truncate_on_insert = settings.local_truncate_on_insert, .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 6491deef440..3836e2c7ec8 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -42,7 +42,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", configuration.getNamespace(), key, - configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName(), configuration.getTypeName()); } From 0b58cbca3836a9dbf2babb9a0c2d7ed89d44b90b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:02:25 +0000 Subject: [PATCH 121/409] Fix submodules --- .gitmodules | 3 +++ contrib/usearch | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/usearch diff --git a/.gitmodules b/.gitmodules index 26778cda171..cdee6a43ad8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -336,6 +336,9 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git +[submodule "contrib/usearch"] + path = contrib/usearch + url = https://github.com/ClickHouse/usearch.git [submodule "contrib/SimSIMD"] path = contrib/SimSIMD url = https://github.com/ashvardanian/SimSIMD.git diff --git a/contrib/usearch b/contrib/usearch new file mode 160000 index 00000000000..e21a5778a0d --- /dev/null +++ b/contrib/usearch @@ -0,0 +1 @@ +Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 From 53bff26f11301781f92a3005c9a08fe176ff0bda Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:09:39 +0000 Subject: [PATCH 122/409] Try to fix submodule --- contrib/libprotobuf-mutator | 2 +- contrib/libunwind | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index 1f95f808306..b922c8ab900 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d +Subproject commit b922c8ab9004ef9944982e4f165e2747b13223fa diff --git a/contrib/libunwind b/contrib/libunwind index a89d904befe..601db0b0e03 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 +Subproject commit 601db0b0e03018c01710470a37703b618f9cf08b From ef1f0e2aafc68d312cfc793aa4652e81e58c199b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:16:09 +0000 Subject: [PATCH 123/409] Fix typo in docs --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 78e076a7427..94468066372 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -20,7 +20,7 @@ CREATE TABLE iceberg_table_azure ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) CREATE TABLE iceberg_table_local - ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) + ENGINE = IcebergLocal(path_to_table, [,format] [,compression_method]) ``` **Engine arguments** From f17b70e3f3bea09dc09c4b22d857fbab4e1417e4 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 14:06:00 +0000 Subject: [PATCH 124/409] Resolve issues with settings --- src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 262 ------------------ .../ObjectStorage/Local/Configuration.cpp | 4 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 4 files changed, 3 insertions(+), 270 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19a59bf7777..6bca8a99f02 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -116,9 +116,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ - M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ - M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6f73ca4059f..815880eb63e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,267 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"}, - {"local_truncate_on_insert", false, false, "Enables or disables truncating the file on each insert in local object storage engine tables"}, - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -350,7 +89,6 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.local_truncate_on_insert, - .create_new_file_on_insert = settings.local_create_new_file_on_insert, + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 77725f8ed18..73410d959e0 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,10 +40,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), - key, - configuration.getTypeName(), - configuration.getTypeName()); + configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); } void resolveSchemaAndFormat( 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 125/409] 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 3972991b1f364540927858e5f45bf519a1bba928 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 15 Aug 2024 23:49:49 +0200 Subject: [PATCH 126/409] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 1eb2e707194..19c8b2f084e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -47,7 +47,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (!future_part->parts.empty()) { source_data_version = future_part->parts[0]->info.getDataVersion(); - is_mutation = (result_part_info.getDataVersion() != source_data_version); + is_mutation = (result_part_info.level == future_part->parts[0]->info.level); WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; From 5ff4d990e189dfee42eb57f567a5ff6313cfa8d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 16 Aug 2024 11:11:11 +0200 Subject: [PATCH 127/409] CI: Auto Releases in prod --- .github/workflows/auto_releases.yml | 76 ++++++++++------------------ .github/workflows/create_release.yml | 2 + tests/ci/auto_release.py | 9 ++++ tests/ci/ci_utils.py | 24 ++++++--- 4 files changed, 54 insertions(+), 57 deletions(-) diff --git a/.github/workflows/auto_releases.yml b/.github/workflows/auto_releases.yml index c159907187c..28483ea136f 100644 --- a/.github/workflows/auto_releases.yml +++ b/.github/workflows/auto_releases.yml @@ -19,13 +19,11 @@ on: jobs: AutoReleaseInfo: - runs-on: [self-hosted, style-checker-aarch64] + runs-on: [self-hosted, release-maker] outputs: data: ${{ steps.info.outputs.AUTO_RELEASE_PARAMS }} dry_run: ${{ steps.info.outputs.DRY_RUN }} steps: - - name: Debug Info - uses: ./.github/actions/debug - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' @@ -36,6 +34,10 @@ jobs: echo "DRY_RUN=true" >> "$GITHUB_ENV" - name: Check out repository code uses: ClickHouse/checkout@v1 + with: + fetch-depth: 0 # full history needed + - name: Debug Info + uses: ./.github/actions/debug - name: Prepare Info id: info run: | @@ -46,12 +48,7 @@ jobs: echo "::endgroup::" { echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_ENV" - { - echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_OUTPUT" echo "DRY_RUN=true" >> "$GITHUB_OUTPUT" @@ -62,48 +59,29 @@ jobs: - name: Clean up uses: ./.github/actions/clean - Release_0: + Releases: needs: AutoReleaseInfo - name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].release_branch }} - if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].ready }} + strategy: + matrix: + release_params: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases }} + max-parallel: 1 + name: Release ${{ matrix.release_params.release_branch }} uses: ./.github/workflows/create_release.yml with: - ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} + ref: ${{ matrix.release_params.commit_sha }} type: patch - dry-run: ${{ needs.AutoReleaseInfo.outputs.dry_run }} -# -# Release_1: -# needs: [AutoReleaseInfo, Release_0] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].ready }} -# uses: ./.github/workflows/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} -# -# Release_2: -# needs: [AutoReleaseInfo, Release_1] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].ready }} -# uses: ./.github/workflow/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} -# -# Release_3: -# needs: [AutoReleaseInfo, Release_2] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].ready }} -# uses: ./.github/workflow/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} + dry-run: ${{ fromJson(needs.AutoReleaseInfo.outputs.dry_run) }} + secrets: + ROBOT_CLICKHOUSE_COMMIT_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} -# - name: Post Slack Message -# if: ${{ !cancelled() }} -# run: | -# cd "$GITHUB_WORKSPACE/tests/ci" -# python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} + PostSlackMessage: + needs: [AutoReleaseInfo] + runs-on: [self-hosted, release-maker] + if: ${{ !cancelled() }} + steps: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + - name: Post + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 1553d689227..1fb6cb60e96 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -47,6 +47,8 @@ concurrency: required: false default: false type: boolean + secrets: + ROBOT_CLICKHOUSE_COMMIT_TOKEN: jobs: CreateRelease: diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 3cc88634004..58cfc833afe 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,4 +1,5 @@ import argparse +import copy import dataclasses import json import os @@ -46,6 +47,7 @@ def parse_args(): MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" +AUTORELEASE_MATRIX_PARAMS = "/tmp/autorelease_params.json" @dataclasses.dataclass @@ -74,6 +76,12 @@ class AutoReleaseInfo: with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + # dump file for GH action matrix that is similar to the file above but with dropped not ready release branches + params = copy.deepcopy(self) + params.releases = [release for release in params.releases if release.ready] + with open(AUTORELEASE_MATRIX_PARAMS, "w", encoding="utf-8") as f: + print(json.dumps(params, indent=2), file=f) + @staticmethod def from_file() -> "AutoReleaseInfo": with open(AUTORELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: @@ -136,6 +144,7 @@ def _prepare(token): commit_ci_status = CI.GH.get_commit_status_by_name( token=token, commit_sha=commit, + # handle old name for old releases status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), ) commit_sha = commit diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index b8778e0cc50..97ab10f1b58 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -102,21 +102,29 @@ class GH: assert len(commit_sha) == 40 assert Utils.is_hex(commit_sha) assert not Utils.is_hex(token) - url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses?per_page={200}" + + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses" headers = { "Authorization": f"token {token}", "Accept": "application/vnd.github.v3+json", } - response = requests.get(url, headers=headers, timeout=5) if isinstance(status_name, str): status_name = (status_name,) - if response.status_code == 200: - assert "next" not in response.links, "Response truncated" - statuses = response.json() - for status in statuses: - if status["context"] in status_name: - return status["state"] # type: ignore + + while url: + response = requests.get(url, headers=headers, timeout=5) + if response.status_code == 200: + statuses = response.json() + for status in statuses: + if status["context"] in status_name: + return status["state"] + + # Check if there is a next page + url = response.links.get("next", {}).get("url") + else: + break + return "" @staticmethod From dfd17cc2d71555de9c42ad6085c35bb3f1372dd1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:23:57 +0000 Subject: [PATCH 128/409] Check for invalid regexp in JSON SKIP REGEXP section --- src/DataTypes/DataTypeObject.cpp | 11 +++++++++++ .../0_stateless/03227_json_invalid_regexp.reference | 0 .../queries/0_stateless/03227_json_invalid_regexp.sql | 4 ++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03227_json_invalid_regexp.reference create mode 100644 tests/queries/0_stateless/03227_json_invalid_regexp.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index d6395155397..11fffd8769b 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -49,6 +49,17 @@ DataTypeObject::DataTypeObject( , max_dynamic_paths(max_dynamic_paths_) , max_dynamic_types(max_dynamic_types_) { + /// Check if regular expressions are valid. + for (const auto & regexp_str : path_regexps_to_skip) + { + re2::RE2::Options options; + /// Don't log errors to stderr. + options.set_log_errors(false); + auto regexp = re2::RE2(regexp_str, options); + if (!regexp.error().empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid regexp '{}': {}", regexp_str, regexp.error()); + } + for (const auto & [typed_path, type] : typed_paths) { for (const auto & path_to_skip : paths_to_skip) diff --git a/tests/queries/0_stateless/03227_json_invalid_regexp.reference b/tests/queries/0_stateless/03227_json_invalid_regexp.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03227_json_invalid_regexp.sql b/tests/queries/0_stateless/03227_json_invalid_regexp.sql new file mode 100644 index 00000000000..734dea1aac6 --- /dev/null +++ b/tests/queries/0_stateless/03227_json_invalid_regexp.sql @@ -0,0 +1,4 @@ +set allow_experimental_json_type = 1; +create table test (json JSON(SKIP REGEXP '[]')) engine=Memory(); -- {serverError BAD_ARGUMENTS} +create table test (json JSON(SKIP REGEXP '+')) engine=Memory(); -- {serverError BAD_ARGUMENTS}; + From 370b6bdc7b6d97f0e697e99ccd06a25e97651406 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:38:30 +0000 Subject: [PATCH 129/409] 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 130/409] 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 4f84c82d6d53ded0adda46aac1db1d345b5ba2eb Mon Sep 17 00:00:00 2001 From: Linh Giang <165205637+linhgiang24@users.noreply.github.com> Date: Fri, 16 Aug 2024 11:02:44 -0600 Subject: [PATCH 131/409] Update grant.md to include POSTGRES privilege Added POSTGRES privilege under the SOURCES category as it seems to be missing. --- docs/en/sql-reference/statements/grant.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 43fa344a16d..6118f4c1d36 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -200,6 +200,7 @@ Hierarchy of privileges: - `JDBC` - `HDFS` - `S3` + - `POSTGRES` - [dictGet](#dictget) - [displaySecretsInShowAndSelect](#displaysecretsinshowandselect) - [NAMED COLLECTION ADMIN](#named-collection-admin) @@ -476,6 +477,7 @@ Allows using external data sources. Applies to [table engines](../../engines/tab - `JDBC`. Level: `GLOBAL` - `HDFS`. Level: `GLOBAL` - `S3`. Level: `GLOBAL` + - `POSTGRES`. Level: `GLOBAL` The `SOURCES` privilege enables use of all the sources. Also you can grant a privilege for each source individually. To use sources, you need additional privileges. From a66db7abc2cced463e8ff9a04530c4647ddf8a22 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Aug 2024 00:05:18 +0200 Subject: [PATCH 132/409] Fix output of clickhouse-test in case of tests timeouts After https://github.com/ClickHouse/ClickHouse/pull/67737 the output will be broken, since in case of timeout it will print to stdout. Let's just capture it and add it to stderr. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3d7e0e922d..24149d46d14 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -30,6 +30,7 @@ import subprocess import sys import traceback import urllib.parse +import io # for crc32 import zlib @@ -39,6 +40,8 @@ from errno import ESRCH from subprocess import PIPE, Popen from time import sleep, time from typing import Dict, List, Optional, Set, Tuple, Union +from contextlib import redirect_stdout + try: import termcolor # type: ignore @@ -1324,9 +1327,13 @@ class TestCase: return None def process_result_impl(self, proc, total_time: float): + kill_output = "" if proc: if proc.returncode is None: - kill_process_group(os.getpgid(proc.pid)) + f = io.StringIO() + with redirect_stdout(f): + kill_process_group(os.getpgid(proc.pid)) + kill_output = f.getvalue() description = "" @@ -1342,7 +1349,7 @@ class TestCase: with open(self.stdout_file, "rb") as stdfd: stdout = str(stdfd.read(), errors="replace", encoding="utf-8") - stderr = "" + stderr = kill_output if os.path.exists(self.stderr_file): with open(self.stderr_file, "rb") as stdfd: stderr += str(stdfd.read(), errors="replace", encoding="utf-8") From 3adfea86539389fd690407b40c2708ab4e19d71a Mon Sep 17 00:00:00 2001 From: Denis Hananein Date: Sat, 17 Aug 2024 06:06:04 +0200 Subject: [PATCH 133/409] Fix #68239 SAMPLE n --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 59f3a299c99..3b148a4971d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -92,11 +92,11 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( /// We will find out how many rows we would have read without sampling. LOG_DEBUG(log, "Preliminary index scan with condition: {}", key_condition.toString()); + MarkRanges exact_ranges; for (const auto & part : parts) { - MarkRanges exact_ranges; - markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log); - for (const auto & range : exact_ranges) + MarkRanges part_ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log); + for (const auto & range : part_ranges) rows_count += part->index_granularity.getRowsCountInRange(range); } From cb8d9a05643d3aac5f410c4eac53124224c63bc8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 17 Aug 2024 20:13:35 +0200 Subject: [PATCH 134/409] fix typo --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index cd6fd9ab839..a03394be226 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4952,7 +4952,7 @@ toIntervalMonth(n) **Arguments** -- `n` — Number of m. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of months. Positive integer number. [Int*](../data-types/int-uint.md). **Returned values** From ae389d14ee65ff5fea3543868b6b161f9fcb806e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 17 Aug 2024 20:42:00 +0200 Subject: [PATCH 135/409] Fix stylecheck --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 54352b88578..e8eab3b4453 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -333,7 +333,7 @@ namespace type = variant_type; type_indexes = {TypeIndex::Variant}; } - + /// If we have only date/datetimes types (Date/DateTime/DateTime64), convert all of them to the common type, /// otherwise, convert all Date, DateTime and DateTime64 to String. void transformDatesAndDateTimes(DataTypes & data_types, TypeIndexesSet & type_indexes) From a8a31ed137df53ccad60a16e09f63edf82a7b9cd Mon Sep 17 00:00:00 2001 From: Denis Hananein Date: Sat, 17 Aug 2024 21:14:25 +0200 Subject: [PATCH 136/409] Add unused --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3b148a4971d..bf12b3966d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -99,6 +99,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( for (const auto & range : part_ranges) rows_count += part->index_granularity.getRowsCountInRange(range); } + UNUSED(exact_ranges); return rows_count; } From d6e170f77704833fa6655820d55090ba18b0b9fe Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Sat, 17 Aug 2024 18:31:11 +0800 Subject: [PATCH 137/409] repeat field is also compound types, ignore it. --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c6167e572df..7b5c29e321f 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -286,6 +286,9 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (!s) continue; + if(s->descr()->schema_node()->is_repeated()) + continue; + auto path = c->path_in_schema()->ToDotVector(); if (path.size() != 1) continue; // compound types not supported From 858f8b502002661584e6153d39a23edc87b49dda Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Sun, 18 Aug 2024 19:11:30 +0800 Subject: [PATCH 138/409] add test and update codes per commit --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- .../02841_parquet_filter_pushdown_bug.reference | 1 + .../02841_parquet_filter_pushdown_bug.sh.sh | 8 ++++++++ .../0_stateless/data_parquet/68131.parquet | Bin 0 -> 289 bytes 4 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference create mode 100755 tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh create mode 100644 tests/queries/0_stateless/data_parquet/68131.parquet diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7b5c29e321f..1f213fef731 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -286,7 +286,7 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (!s) continue; - if(s->descr()->schema_node()->is_repeated()) + if (s->descr()->schema_node()->is_repeated()) continue; auto path = c->path_in_schema()->ToDotVector(); diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference new file mode 100644 index 00000000000..6ed63af507a --- /dev/null +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference @@ -0,0 +1 @@ +[1,2] diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh new file mode 100755 index 00000000000..58eb207b6e6 --- /dev/null +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select f from file('$CURDIR/data_parquet/68131.parquet', Parquet, 'f Array(Int32)')" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_parquet/68131.parquet b/tests/queries/0_stateless/data_parquet/68131.parquet new file mode 100644 index 0000000000000000000000000000000000000000..169f6152003db164c78e33cd69205caa33f906b5 GIT binary patch literal 289 zcmXAl!D_=W42Bgqgq#M0O4Q&(E)5xMp|QKBg Date: Sun, 18 Aug 2024 15:10:35 +0000 Subject: [PATCH 139/409] Add explicit session_timezone to UTC --- tests/queries/0_stateless/03222_datetime64_small_value_const.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql index af06a622f8d..39266ba7992 100644 --- a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -1,4 +1,5 @@ -- Tags: shard +set session_timezone = 'UTC'; -- don't randomize the session timezone select *, (select toDateTime64(0, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; select *, (select toDateTime64(5, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; From 1049e366534635a510e9f0c769b5635a073a0c1c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 18 Aug 2024 23:35:20 +0200 Subject: [PATCH 140/409] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 19c8b2f084e..2465222ae6c 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -30,8 +30,10 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (result_part_name != result_part_info.getPartNameV1()) format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; + size_t normal_parts_count = 0; for (const auto & source_part : future_part->parts) { + normal_parts_count += !source_part->getParentPart(); if (!source_part->getParentPart() && !result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); @@ -54,7 +56,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta part->partition.serializeText(part->storage, out, {}); } - if (is_mutation && future_part->parts.size() != 1) + if (is_mutation && normal_parts_count != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}", future_part->parts.size(), result_part_info.getPartNameV1()); thread_group = ThreadGroup::createForBackgroundProcess(context); From 8eb922036e5b7caa36c1b904b43fdaee8e45acaa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 19 Aug 2024 12:45:15 +0800 Subject: [PATCH 141/409] change as request --- .../functions/string-replace-functions.md | 16 +- src/Functions/overlay.cpp | 140 +++++------ .../0_stateless/03205_overlay.reference | 230 +++++------------- tests/queries/0_stateless/03205_overlay.sql | 132 +++++----- .../0_stateless/03206_overlay_utf8.reference | 168 ------------- .../0_stateless/03206_overlay_utf8.sql | 60 ----- 6 files changed, 203 insertions(+), 543 deletions(-) delete mode 100644 tests/queries/0_stateless/03206_overlay_utf8.reference delete mode 100644 tests/queries/0_stateless/03206_overlay_utf8.sql diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index d086c9ee64b..408a896e607 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -251,24 +251,24 @@ select printf('%%%s %s %d', 'Hello', 'World', 2024); ## overlay -Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. **Syntax** ```sql -overlay(s, replace, position[, length]) +overlay(s, replace, offset[, length]) ``` **Parameters** - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int-uint.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). - `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** -- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. +- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. **Example** @@ -286,26 +286,26 @@ Result: ## overlayUTF8 -Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. **Syntax** ```sql -overlayUTF8(s, replace, position[, length]) +overlayUTF8(s, replace, offset[, length]) ``` **Parameters** - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int-uint.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). - `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** -- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. +- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. **Example** diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 094da27a71d..73ca0acbb8e 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -17,13 +17,13 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -using namespace GatherUtils; - namespace { /// If 'is_utf8' - measure offset and length in code points instead of bytes. -/// Syntax: overlay(input, replace, offset[, length]) +/// Syntax: +/// - overlay(input, replace, offset[, length]) +/// - overlayUTF8(input, replace, offset[, length]) - measure offset and length in code points instead of bytes template class FunctionOverlay : public IFunction { @@ -37,63 +37,39 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const size_t number_of_arguments = arguments.size(); - if (number_of_arguments < 3 || number_of_arguments > 4) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: " - "passed {}, should be 3 or 4", - getName(), - number_of_arguments); + FunctionArgumentDescriptors mandatory_args{ + {"input", static_cast(&isString), nullptr, "String"}, + {"replace", static_cast(&isString), nullptr, "String"}, + {"offset", static_cast(&isNativeInteger), nullptr, "(U)Int8/16/32/64"}, + }; - /// first argument is string - if (!isString(arguments[0])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {}, expected String", - arguments[0]->getName(), - getName()); + FunctionArgumentDescriptors optional_args{ + {"length", static_cast(&isNativeInteger), nullptr, "(U)Int8/16/32/64"}, + }; - /// second argument is string - if (!isString(arguments[1])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}, expected String", - arguments[1]->getName(), - getName()); - - if (!isNativeNumber(arguments[2])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {}, expected (U)Int8|16|32|64", - arguments[2]->getName(), - getName()); - - if (number_of_arguments == 4 && !isNativeNumber(arguments[3])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}, expected (U)Int8|16|32|64", - arguments[3]->getName(), - getName()); + validateFunctionArguments(*this, arguments, mandatory_args, optional_args); return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (input_rows_count == 0) + return ColumnString::create(); + const size_t number_of_arguments = arguments.size(); - bool three_args = number_of_arguments == 3; + bool has_three_args = number_of_arguments == 3; ColumnPtr column_offset = arguments[2].column; ColumnPtr column_length; - if (!three_args) + if (!has_three_args) column_length = arguments[3].column; const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); const ColumnConst * column_length_const = nullptr; - if (!three_args) + if (!has_three_args) column_length_const = checkAndGetColumn(column_length.get()); bool offset_is_const = false; @@ -126,7 +102,7 @@ public: if (column_input_const) { StringRef input = column_input_const->getDataAt(0); - res_data.reserve(input.size * input_rows_count); + res_data.reserve((input.size + 1) * input_rows_count); } else { @@ -135,8 +111,8 @@ public: const auto * column_replace_const = checkAndGetColumn(column_replace.get()); const auto * column_replace_string = checkAndGetColumn(column_replace.get()); - bool input_is_const = column_input_const != nullptr; - bool replace_is_const = column_replace_const != nullptr; + bool input_is_const = (column_input_const != nullptr); + bool replace_is_const = (column_replace_const != nullptr); #define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ if (input_is_const && replace_is_const) \ @@ -150,8 +126,9 @@ public: length, \ res_data, \ res_offsets); \ - else if (input_is_const) \ + else if (input_is_const && !replace_is_const) \ constantVector( \ + input_rows_count, \ column_input_const->getDataAt(0), \ column_replace_string->getChars(), \ column_replace_string->getOffsets(), \ @@ -161,8 +138,9 @@ public: length, \ res_data, \ res_offsets); \ - else if (replace_is_const) \ + else if (!input_is_const && replace_is_const) \ vectorConstant( \ + input_rows_count, \ column_input_string->getChars(), \ column_input_string->getOffsets(), \ column_replace_const->getDataAt(0), \ @@ -174,6 +152,7 @@ public: res_offsets); \ else \ vectorVector( \ + input_rows_count, \ column_input_string->getChars(), \ column_input_string->getOffsets(), \ column_replace_string->getChars(), \ @@ -185,7 +164,7 @@ public: res_data, \ res_offsets); - if (three_args) + if (has_three_args) { if (offset_is_const) { @@ -251,7 +230,7 @@ private: return bytes; } - template + template void constantConstant( size_t rows, const StringRef & input, @@ -263,7 +242,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { constantConstant( rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -277,12 +256,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (three_args) + else if constexpr (has_three_args) { valid_length = replace_size; } @@ -300,7 +279,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!three_args && !length_is_const) + if constexpr (!has_three_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; @@ -331,10 +310,10 @@ private: } else { - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator @@ -363,8 +342,9 @@ private: } } - template + template void vectorConstant( + size_t rows, const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, const StringRef & replace, @@ -375,27 +355,26 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { vectorConstant( - input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + rows, input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (three_args) + else if constexpr (has_three_args) { valid_length = replace_size; } - size_t rows = input_offsets.size(); Int64 offset = 0; // start from 1, maybe negative size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; @@ -415,7 +394,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!three_args && !length_is_const) + if constexpr (!has_three_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; @@ -449,9 +428,9 @@ private: { const auto * input_begin = &input_data[input_offset]; const auto * input_end = &input_data[input_offset + input_bytes]; - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator @@ -480,8 +459,9 @@ private: } } - template + template void constantVector( + size_t rows, const StringRef & input, const ColumnString::Chars & replace_data, const ColumnString::Offsets & replace_offsets, @@ -492,10 +472,10 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { constantVector( - input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); + rows, input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -506,13 +486,12 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - size_t rows = replace_offsets.size(); const auto * input_begin = reinterpret_cast(input.data); const auto * input_end = reinterpret_cast(input.data + input.size); Int64 offset = 0; // start from 1, maybe negative @@ -529,7 +508,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (three_args) + if constexpr (has_three_args) { valid_length = replace_size; } @@ -564,9 +543,9 @@ private: } else { - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -594,8 +573,9 @@ private: } } - template + template void vectorVector( + size_t rows, const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, const ColumnString::Chars & replace_data, @@ -607,9 +587,10 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { vectorVector( + rows, input_data, input_offsets, replace_data, @@ -625,13 +606,12 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - size_t rows = input_offsets.size(); Int64 offset = 0; // start from 1, maybe negative size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; @@ -655,7 +635,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (three_args) + if constexpr (has_three_args) { valid_length = replace_size; } @@ -693,9 +673,9 @@ private: { const auto * input_begin = &input_data[input_offset]; const auto * input_end = &input_data[input_offset + input_bytes]; - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator res_data.resize(new_res_size); diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference index 9e79db2e131..383a26986d6 100644 --- a/tests/queries/0_stateless/03205_overlay.reference +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -1,168 +1,62 @@ -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL +Negative test of overlay +Positive test 1 with various combinations of const/non-const columns +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Positive test 2 with various combinations of const/non-const columns +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Spark_SQL Spark_SQL和CH +Positive test 3 with various combinations of const/non-const columns +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Positive test 4 with various combinations of const/non-const columns +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Positive test 5 with various combinations of const/non-const columns +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH +Structured SQL Structured SQL和CH diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql index b131312c934..4fd0791521d 100644 --- a/tests/queries/0_stateless/03205_overlay.sql +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -1,60 +1,74 @@ -SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT 'Negative test of overlay'; +SELECT overlay('hello', 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay('hello', 'world'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay('hello', 'world', 2, 3, 'extra'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay(123, 'world', 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 456, 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 'world', 'two', 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 'world', 2, 'three'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT overlay('Spark SQL', '_', 6) from numbers(3); -SELECT overlay(materialize('Spark SQL'), '_', 6) from numbers(3); -SELECT overlay('Spark SQL', materialize('_'), 6) from numbers(3); -SELECT overlay('Spark SQL', '_', materialize(6)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('_'), 6) from numbers(3); -SELECT overlay(materialize('Spark SQL'), '_', materialize(6)) from numbers(3); -SELECT overlay('Spark SQL', materialize('_'), materialize(6)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)) from numbers(3); - -SELECT overlay('Spark SQL', 'CORE', 7) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'CORE', 7) from numbers(3); -SELECT overlay('Spark SQL', materialize('CORE'), 7) from numbers(3); -SELECT overlay('Spark SQL', 'CORE', materialize(7)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)) from numbers(3); -SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)) from numbers(3); - -SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlay('Spark SQL', 'tructured', 2, 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), 2, 4) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', materialize(2), 4) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); +SELECT 'Positive test 1 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)); + +SELECT 'Positive test 2 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', '_', 6), overlayUTF8('Spark SQL和CH', '_', 6); +SELECT overlay(materialize('Spark SQL'), '_', 6), overlayUTF8(materialize('Spark SQL和CH'), '_', 6); +SELECT overlay('Spark SQL', materialize('_'), 6), overlayUTF8('Spark SQL和CH', materialize('_'), 6); +SELECT overlay('Spark SQL', '_', materialize(6)), overlayUTF8('Spark SQL和CH', '_', materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), 6), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), 6); +SELECT overlay(materialize('Spark SQL'), '_', materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), '_', materialize(6)); +SELECT overlay('Spark SQL', materialize('_'), materialize(6)), overlayUTF8('Spark SQL和CH', materialize('_'), materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), materialize(6)); + +SELECT 'Positive test 3 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'CORE', 7), overlayUTF8('Spark SQL和CH', 'CORE', 7); +SELECT overlay(materialize('Spark SQL'), 'CORE', 7), overlayUTF8(materialize('Spark SQL和CH'), 'CORE', 7); +SELECT overlay('Spark SQL', materialize('CORE'), 7), overlayUTF8('Spark SQL和CH', materialize('CORE'), 7); +SELECT overlay('Spark SQL', 'CORE', materialize(7)), overlayUTF8('Spark SQL和CH', 'CORE', materialize(7)); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7), overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), 7); +SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)), overlayUTF8(materialize('Spark SQL和CH'), 'CORE', materialize(7)); +SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)), overlayUTF8('Spark SQL和CH', materialize('CORE'), materialize(7)); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)), overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), materialize(7)); + +SELECT 'Positive test 4 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)); + +SELECT 'Positive test 5 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'tructured', 2, 4), overlayUTF8('Spark SQL和CH', 'tructured', 2, 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, 4); +SELECT overlay('Spark SQL', materialize('tructured'), 2, 4), overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, 4); +SELECT overlay('Spark SQL', 'tructured', materialize(2), 4), overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), 4); +SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)), overlayUTF8('Spark SQL和CH', 'tructured', 2, materialize(4)); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4), overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), 2, 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', materialize(2), 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, materialize(4)); +SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4), overlayUTF8('Spark SQL和CH', materialize('tructured'), materialize(2), 4); +SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)), overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, materialize(4)); +SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)), overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), materialize(4)); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)), overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), materialize(2), materialize(4)); diff --git a/tests/queries/0_stateless/03206_overlay_utf8.reference b/tests/queries/0_stateless/03206_overlay_utf8.reference deleted file mode 100644 index 19878c97184..00000000000 --- a/tests/queries/0_stateless/03206_overlay_utf8.reference +++ /dev/null @@ -1,168 +0,0 @@ -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark_SQL和CH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Spark ANSI SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH -Structured SQL和CH diff --git a/tests/queries/0_stateless/03206_overlay_utf8.sql b/tests/queries/0_stateless/03206_overlay_utf8.sql deleted file mode 100644 index 00b756c8b5b..00000000000 --- a/tests/queries/0_stateless/03206_overlay_utf8.sql +++ /dev/null @@ -1,60 +0,0 @@ -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlayUTF8('Spark SQL和CH', '_', 6) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), '_', 6) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('_'), 6) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', '_', materialize(6)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), 6) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), '_', materialize(6)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('_'), materialize(6)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), materialize(6)) from numbers(3); - -SELECT overlayUTF8('Spark SQL和CH', 'CORE', 7) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'CORE', 7) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('CORE'), 7) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'CORE', materialize(7)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), 7) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'CORE', materialize(7)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('CORE'), materialize(7)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), materialize(7)) from numbers(3); - -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlayUTF8('Spark SQL和CH', 'tructured', 2, 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, 4) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, 4) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), 4) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), 2, 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', materialize(2), 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), materialize(2), 4) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, materialize(4)) from numbers(3); -SELECT overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), materialize(4)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); From 845468b215b0565c1a71c7a5ac339940af1e95ef Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 17:56:12 +0800 Subject: [PATCH 142/409] first commit --- src/Common/examples/CMakeLists.txt | 5 + src/Common/examples/utf8_upper_lower.cpp | 27 ++ src/Functions/LowerUpperImpl.h | 1 - src/Functions/LowerUpperUTF8Impl.h | 283 +++--------------- src/Functions/initcapUTF8.cpp | 3 +- src/Functions/lowerUTF8.cpp | 25 +- src/Functions/upperUTF8.cpp | 24 +- .../00170_lower_upper_utf8.reference | 4 + .../0_stateless/00170_lower_upper_utf8.sql | 11 + .../00233_position_function_family.sql | 3 + .../0_stateless/00761_lower_utf8_bug.sql | 3 + .../0_stateless/01278_random_string_utf8.sql | 3 + .../0_stateless/01431_utf8_ubsan.reference | 4 +- .../queries/0_stateless/01431_utf8_ubsan.sql | 3 + .../0_stateless/01590_countSubstrings.sql | 3 + ...71_lower_upper_utf8_row_overlaps.reference | 4 +- .../02071_lower_upper_utf8_row_overlaps.sql | 3 + ...new_functions_must_be_documented.reference | 2 - .../02514_if_with_lazy_low_cardinality.sql | 3 + .../0_stateless/02807_lower_utf8_msan.sql | 3 + tests/queries/0_stateless/03015_peder1001.sql | 3 + 21 files changed, 159 insertions(+), 261 deletions(-) create mode 100644 src/Common/examples/utf8_upper_lower.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 69580d4ad0e..8383e80d09d 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -92,3 +92,8 @@ endif() clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp) target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io clickhouse_common_config) + +if (TARGET ch_contrib::icu) + clickhouse_add_executable (utf8_upper_lower utf8_upper_lower.cpp) + target_link_libraries (utf8_upper_lower PRIVATE ch_contrib::icu) +endif () diff --git a/src/Common/examples/utf8_upper_lower.cpp b/src/Common/examples/utf8_upper_lower.cpp new file mode 100644 index 00000000000..826e1763105 --- /dev/null +++ b/src/Common/examples/utf8_upper_lower.cpp @@ -0,0 +1,27 @@ +#include +#include + +std::string utf8_to_lower(const std::string & input) +{ + icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); + unicodeInput.toLower(); + std::string output; + unicodeInput.toUTF8String(output); + return output; +} + +std::string utf8_to_upper(const std::string & input) +{ + icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); + unicodeInput.toUpper(); + std::string output; + unicodeInput.toUTF8String(output); + return output; +} + +int main() +{ + std::string input = "ır"; + std::cout << "upper:" << utf8_to_upper(input) << std::endl; + return 0; +} diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index d463ef96e16..a52703d10c8 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -1,7 +1,6 @@ #pragma once #include - namespace DB { diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index eedabca5b22..5da085f48e5 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -1,15 +1,14 @@ #pragma once + +#include "config.h" + +#if USE_ICU + #include #include -#include -#include +#include +#include #include -#include - -#ifdef __SSE2__ -#include -#endif - namespace DB { @@ -19,71 +18,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -/// xor or do nothing -template -UInt8 xor_or_identity(const UInt8 c, const int mask) -{ - return c ^ mask; -} - -template <> -inline UInt8 xor_or_identity(const UInt8 c, const int) -{ - return c; -} - -/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array -template -inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) -{ - if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// ѐёђѓєѕіїјљњћќѝўџ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// А-П - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) - { - /// а-п - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) - { - /// Р-Я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// р-я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } -} - - -/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case. - * Note: It is assumed that after the character is converted to another case, - * the length of its multibyte sequence in UTF-8 does not change. - * Otherwise, the behavior is undefined. - */ -template +template struct LowerUpperUTF8Impl { static void vector( @@ -103,180 +38,46 @@ struct LowerUpperUTF8Impl return; } - res_data.resize_exact(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), offsets, res_data.data()); + res_data.resize(data.size()); + res_offsets.resize_exact(offsets.size()); + + String output; + size_t curr_offset = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); + size_t size = offsets[i] - offsets[i - 1]; + + icu::UnicodeString input(data_start, static_cast(size), "UTF-8"); + if constexpr (upper) + input.toUpper(); + else + input.toLower(); + + output.clear(); + input.toUTF8String(output); + + /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first + /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this + /// case, the behavior is also reasonable. + const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); + size_t valid_size = res_end ? res_end - output.data() + 1 : 0; + + res_data.resize(curr_offset + valid_size + 1); + memcpy(&res_data[curr_offset], output.data(), valid_size); + res_data[curr_offset + valid_size] = 0; + + curr_offset += valid_size + 1; + res_offsets[i] = curr_offset; + } } static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument"); } - - /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. - * `src` and `dst` are incremented by corresponding sequence lengths. */ - static bool toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst, bool partial) - { - if (src[0] <= ascii_upper_bound) - { - if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) - *dst++ = *src++ ^ flip_case_mask; - else - *dst++ = *src++; - } - else if (src + 1 < src_end - && ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu)))) - { - cyrillic_to_case(src, dst); - } - else if (src + 1 < src_end && src[0] == 0xC2u) - { - /// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF - *dst++ = *src++; - *dst++ = *src++; - } - else if (src + 2 < src_end && src[0] == 0xE2u) - { - /// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF - *dst++ = *src++; - *dst++ = *src++; - *dst++ = *src++; - } - else - { - size_t src_sequence_length = UTF8::seqLength(*src); - /// In case partial buffer was passed (due to SSE optimization) - /// we cannot convert it with current src_end, but we may have more - /// bytes to convert and eventually got correct symbol. - if (partial && src_sequence_length > static_cast(src_end - src)) - return false; - - auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src); - if (src_code_point) - { - int dst_code_point = to_case(*src_code_point); - if (dst_code_point > 0) - { - size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src); - assert(dst_sequence_length <= 4); - - /// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8. - /// As an example, this happens for ß and ẞ. - if (dst_sequence_length == src_sequence_length) - { - src += dst_sequence_length; - dst += dst_sequence_length; - return true; - } - } - } - - *dst = *src; - ++dst; - ++src; - } - - return true; - } - -private: - static constexpr auto ascii_upper_bound = '\x7f'; - static constexpr auto flip_case_mask = 'A' ^ 'a'; - - static void array(const UInt8 * src, const UInt8 * src_end, const ColumnString::Offsets & offsets, UInt8 * dst) - { - const auto * offset_it = offsets.begin(); - const UInt8 * begin = src; - -#ifdef __SSE2__ - static constexpr auto bytes_sse = sizeof(__m128i); - - /// If we are before this position, we can still read at least bytes_sse. - const auto * src_end_sse = src_end - bytes_sse + 1; - - /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) - const auto v_zero = _mm_setzero_si128(); - const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); - const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); - const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); - - while (src < src_end_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(src)); - - /// check for ASCII - const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero); - const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii); - - /// ASCII - if (mask_is_not_ascii == 0) - { - const auto is_not_case - = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); - const auto mask_is_not_case = _mm_movemask_epi8(is_not_case); - - /// everything in correct case ASCII - if (mask_is_not_case == 0) - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars); - else - { - /// ASCII in mixed case - /// keep `flip_case_mask` only where necessary, zero out elsewhere - const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); - - /// flip case by applying calculated mask - const auto cased_chars = _mm_xor_si128(chars, xor_mask); - - /// store result back to destination - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); - } - - src += bytes_sse; - dst += bytes_sse; - } - else - { - /// UTF-8 - - /// Find the offset of the next string after src - size_t offset_from_begin = src - begin; - while (offset_from_begin >= *offset_it) - ++offset_it; - - /// Do not allow one row influence another (since row may have invalid sequence, and break the next) - const UInt8 * row_end = begin + *offset_it; - chassert(row_end >= src); - const UInt8 * expected_end = std::min(src + bytes_sse, row_end); - - while (src < expected_end) - { - if (!toCase(src, expected_end, dst, /* partial= */ true)) - { - /// Fallback to handling byte by byte. - src_end_sse = src; - break; - } - } - } - } - - /// Find the offset of the next string after src - size_t offset_from_begin = src - begin; - while (offset_it != offsets.end() && offset_from_begin >= *offset_it) - ++offset_it; -#endif - - /// handle remaining symbols, row by row (to avoid influence of bad UTF8 symbols from one row, to another) - while (src < src_end) - { - const UInt8 * row_end = begin + *offset_it; - chassert(row_end >= src); - - while (src < row_end) - toCase(src, row_end, dst, /* partial= */ false); - ++offset_it; - } - } }; } + +#endif diff --git a/src/Functions/initcapUTF8.cpp b/src/Functions/initcapUTF8.cpp index 282d846094e..004586dce26 100644 --- a/src/Functions/initcapUTF8.cpp +++ b/src/Functions/initcapUTF8.cpp @@ -1,9 +1,8 @@ #include #include -#include #include #include - +#include namespace DB { diff --git a/src/Functions/lowerUTF8.cpp b/src/Functions/lowerUTF8.cpp index 7adb0069121..e2f7cb84730 100644 --- a/src/Functions/lowerUTF8.cpp +++ b/src/Functions/lowerUTF8.cpp @@ -1,9 +1,10 @@ -#include +#include "config.h" + +#if USE_ICU + +#include #include #include -#include -#include - namespace DB { @@ -15,13 +16,25 @@ struct NameLowerUTF8 static constexpr auto name = "lowerUTF8"; }; -using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; +using FunctionLowerUTF8 = FunctionStringToString, NameLowerUTF8>; } REGISTER_FUNCTION(LowerUTF8) { - factory.registerFunction(); + FunctionDocumentation::Description description + = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; + FunctionDocumentation::Syntax syntax = "lowerUTF8(input)"; + FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; + FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; + FunctionDocumentation::Examples examples = { + {"first", "SELECT lowerUTF8('München') as Lowerutf8;", "münchen"}, + }; + FunctionDocumentation::Categories categories = {"String"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); } } + +#endif diff --git a/src/Functions/upperUTF8.cpp b/src/Functions/upperUTF8.cpp index 659e67f0ef3..ef26430331f 100644 --- a/src/Functions/upperUTF8.cpp +++ b/src/Functions/upperUTF8.cpp @@ -1,8 +1,10 @@ +#include "config.h" + +#if USE_ICU + +#include #include #include -#include -#include - namespace DB { @@ -14,13 +16,25 @@ struct NameUpperUTF8 static constexpr auto name = "upperUTF8"; }; -using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; +using FunctionUpperUTF8 = FunctionStringToString, NameUpperUTF8>; } REGISTER_FUNCTION(UpperUTF8) { - factory.registerFunction(); + FunctionDocumentation::Description description + = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; + FunctionDocumentation::Syntax syntax = "upperUTF8(input)"; + FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; + FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; + FunctionDocumentation::Examples examples = { + {"first", "SELECT upperUTF8('München') as Upperutf8;", "MÜNCHEN"}, + }; + FunctionDocumentation::Categories categories = {"String"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); } } + +#endif diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.reference b/tests/queries/0_stateless/00170_lower_upper_utf8.reference index f202cb75513..3c644f22b9b 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.reference +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.reference @@ -22,3 +22,7 @@ 1 1 1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.sql b/tests/queries/0_stateless/00170_lower_upper_utf8.sql index 4caba2033ff..85b6c5c6095 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.sql +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + select lower('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lowerUTF8('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lower('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa'; @@ -27,3 +30,11 @@ select sum(lower(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaАБВ select sum(upper(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; select sum(lowerUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaабвгaaaaaaaa')) = count() from system.one array join range(16384) as n; select sum(upperUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; + +-- Turkish language +select upperUTF8('ır') = 'IR'; +select lowerUTF8('ır') = 'ır'; + +-- German language +select upper('öäüß') = 'öäüß'; +select lower('ÖÄÜẞ') = 'ÖÄÜẞ'; diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index dd7394bc39a..d6668cb7ba4 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SET send_logs_level = 'fatal'; select 1 = position('', ''); diff --git a/tests/queries/0_stateless/00761_lower_utf8_bug.sql b/tests/queries/0_stateless/00761_lower_utf8_bug.sql index de20b894331..a0ab55edc15 100644 --- a/tests/queries/0_stateless/00761_lower_utf8_bug.sql +++ b/tests/queries/0_stateless/00761_lower_utf8_bug.sql @@ -1 +1,4 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT lowerUTF8('\xF0') = lowerUTF8('\xF0'); diff --git a/tests/queries/0_stateless/01278_random_string_utf8.sql b/tests/queries/0_stateless/01278_random_string_utf8.sql index da2dc48c3e1..290d6a0c759 100644 --- a/tests/queries/0_stateless/01278_random_string_utf8.sql +++ b/tests/queries/0_stateless/01278_random_string_utf8.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT randomStringUTF8('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT lengthUTF8(randomStringUTF8(100)); SELECT toTypeName(randomStringUTF8(10)); diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.reference b/tests/queries/0_stateless/01431_utf8_ubsan.reference index c98c950d535..dc785e57851 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.reference +++ b/tests/queries/0_stateless/01431_utf8_ubsan.reference @@ -1,2 +1,2 @@ -FF -FF +EFBFBD +EFBFBD diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.sql b/tests/queries/0_stateless/01431_utf8_ubsan.sql index d6a299225b1..3a28e023805 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.sql +++ b/tests/queries/0_stateless/01431_utf8_ubsan.sql @@ -1,2 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT hex(lowerUTF8('\xFF')); SELECT hex(upperUTF8('\xFF')); diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index b38cbb7d188..5ec4f412d7f 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + -- -- countSubstrings -- diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index a3bac432482..deabef61a88 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; -1,"0xE2","0xE2" +1,"�","�" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -1,"0xE2","0xE2" +1,"�","�" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql index 8ca0a3f5f75..d175e0659d0 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + drop table if exists utf8_overlap; create table utf8_overlap (str String) engine=Memory(); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index c39f1fb1ce9..0980e25b70f 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -416,7 +416,6 @@ logTrace lowCardinalityIndices lowCardinalityKeys lower -lowerUTF8 makeDate makeDate32 makeDateTime @@ -897,7 +896,6 @@ tupleToNameValuePairs unbin unhex upper -upperUTF8 uptime validateNestedArraySizes version diff --git a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql index 80e3c0a9ece..b169cfd0ab9 100644 --- a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql +++ b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + create table if not exists t (`arr.key` Array(LowCardinality(String)), `arr.value` Array(LowCardinality(String))) engine = Memory; insert into t (`arr.key`, `arr.value`) values (['a'], ['b']); select if(true, if(lowerUTF8(arr.key) = 'a', 1, 2), 3) as x from t left array join arr; diff --git a/tests/queries/0_stateless/02807_lower_utf8_msan.sql b/tests/queries/0_stateless/02807_lower_utf8_msan.sql index e9eb18bf615..95f224577f7 100644 --- a/tests/queries/0_stateless/02807_lower_utf8_msan.sql +++ b/tests/queries/0_stateless/02807_lower_utf8_msan.sql @@ -1,2 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT lowerUTF8(arrayJoin(['©--------------------------------------', '©--------------------'])) ORDER BY 1; SELECT upperUTF8(materialize('aaaaАБВГaaaaaaaaaaaaАБВГAAAAaaAA')) FROM numbers(2); diff --git a/tests/queries/0_stateless/03015_peder1001.sql b/tests/queries/0_stateless/03015_peder1001.sql index 810503207f2..df8e4db1536 100644 --- a/tests/queries/0_stateless/03015_peder1001.sql +++ b/tests/queries/0_stateless/03015_peder1001.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + DROP TABLE IF EXISTS test_data; CREATE TABLE test_data From ef624e9b22a81c2cd7857224d6fe30c7ea4eed81 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 17:58:54 +0800 Subject: [PATCH 143/409] remote icu contrib --- .gitmodules | 3 --- contrib/icu | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/icu diff --git a/.gitmodules b/.gitmodules index cdee6a43ad8..4e0e341ea3b 100644 --- a/.gitmodules +++ b/.gitmodules @@ -106,9 +106,6 @@ [submodule "contrib/icudata"] path = contrib/icudata url = https://github.com/ClickHouse/icudata -[submodule "contrib/icu"] - path = contrib/icu - url = https://github.com/unicode-org/icu [submodule "contrib/flatbuffers"] path = contrib/flatbuffers url = https://github.com/ClickHouse/flatbuffers diff --git a/contrib/icu b/contrib/icu deleted file mode 160000 index 7750081bda4..00000000000 --- a/contrib/icu +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 From db1ba8d5385a85925648dfa8c131021022917024 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 18:04:15 +0800 Subject: [PATCH 144/409] add submodule contrib/icu from clickhouse --- .gitmodules | 3 +++ contrib/icu | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/icu diff --git a/.gitmodules b/.gitmodules index 4e0e341ea3b..53ebde0cd3b 100644 --- a/.gitmodules +++ b/.gitmodules @@ -106,6 +106,9 @@ [submodule "contrib/icudata"] path = contrib/icudata url = https://github.com/ClickHouse/icudata +[submodule "contrib/icu"] + path = contrib/icu + url = https://github.com/ClickHouse/icu [submodule "contrib/flatbuffers"] path = contrib/flatbuffers url = https://github.com/ClickHouse/flatbuffers diff --git a/contrib/icu b/contrib/icu new file mode 160000 index 00000000000..4216173eeeb --- /dev/null +++ b/contrib/icu @@ -0,0 +1 @@ +Subproject commit 4216173eeeb39c1d4caaa54a68860e800412d273 From 29a0161cdfeeed739578af279092e7694dd7fbda Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 19 Aug 2024 18:25:01 +0800 Subject: [PATCH 145/409] change as request --- .../functions/string-replace-functions.md | 5 ++-- src/Functions/overlay.cpp | 24 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 408a896e607..55e97688b18 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -251,7 +251,8 @@ select printf('%%%s %s %d', 'Hello', 'World', 2024); ## overlay -Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. + **Syntax** @@ -286,7 +287,7 @@ Result: ## overlayUTF8 -Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of characters removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of characters is removed. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 73ca0acbb8e..20988c775a5 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -11,12 +11,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int ILLEGAL_TYPE_OF_ARGUMENT; -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - namespace { @@ -708,8 +702,20 @@ private: REGISTER_FUNCTION(Overlay) { - factory.registerFunction>({}, FunctionFactory::Case::Insensitive); - factory.registerFunction>({}, FunctionFactory::Case::Sensitive); -} + factory.registerFunction>( + {.description = R"( +Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. +)", + .categories{"String"}}, + FunctionFactory::Case::Insensitive); + factory.registerFunction>( + {.description = R"( +Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +)", + .categories{"String"}}, + FunctionFactory::Case::Sensitive); +} } From 8653c547c9e5543b46cead484eb446725ec737d6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 19 Aug 2024 18:50:04 +0800 Subject: [PATCH 146/409] change as request --- .../0_stateless/03205_overlay.reference | 88 ++++++++++--------- tests/queries/0_stateless/03205_overlay.sql | 72 +++++---------- 2 files changed, 70 insertions(+), 90 deletions(-) diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference index 383a26986d6..67a699944e0 100644 --- a/tests/queries/0_stateless/03205_overlay.reference +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -1,22 +1,5 @@ Negative test of overlay -Positive test 1 with various combinations of const/non-const columns -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Spark ANSI SQL Spark ANSI SQL和CH -Positive test 2 with various combinations of const/non-const columns +Test with 3 arguments and various combinations of const/non-const columns Spark_SQL Spark_SQL和CH Spark_SQL Spark_SQL和CH Spark_SQL Spark_SQL和CH @@ -25,16 +8,7 @@ Spark_SQL Spark_SQL和CH Spark_SQL Spark_SQL和CH Spark_SQL Spark_SQL和CH Spark_SQL Spark_SQL和CH -Positive test 3 with various combinations of const/non-const columns -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Spark CORE Spark CORECH -Positive test 4 with various combinations of const/non-const columns +Test with 4 arguments and various combinations of const/non-const columns Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH @@ -47,16 +21,48 @@ Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH -Positive test 5 with various combinations of const/non-const columns -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH -Structured SQL Structured SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Spark ANSI SQL Spark ANSI SQL和CH +Test with different offset values +-12 _park SQL _park SQL和CH +-11 _park SQL S_ark SQL和CH +-10 _park SQL Sp_rk SQL和CH +-9 _park SQL Spa_k SQL和CH +-8 S_ark SQL Spar_ SQL和CH +-7 Sp_rk SQL Spark_SQL和CH +-6 Spa_k SQL Spark _QL和CH +-5 Spar_ SQL Spark S_L和CH +-4 Spark_SQL Spark SQ_和CH +-3 Spark _QL Spark SQL_CH +-2 Spark S_L Spark SQL和_H +-1 Spark SQ_ Spark SQL和C_ +0 Spark SQL_ Spark SQL和CH_ +1 _park SQL _park SQL和CH +2 S_ark SQL S_ark SQL和CH +3 Sp_rk SQL Sp_rk SQL和CH +4 Spa_k SQL Spa_k SQL和CH +5 Spar_ SQL Spar_ SQL和CH +6 Spark_SQL Spark_SQL和CH +7 Spark _QL Spark _QL和CH +8 Spark S_L Spark S_L和CH +9 Spark SQ_ Spark SQ_和CH +10 Spark SQL_ Spark SQL_CH +11 Spark SQL_ Spark SQL和_H +12 Spark SQL_ Spark SQL和C_ +13 Spark SQL_ Spark SQL和CH_ +Test with different length values +-1 Spark ANSI Spark ANSI H +0 Spark ANSI SQL Spark ANSI SQL和CH +1 Spark ANSI QL Spark ANSI QL和CH +2 Spark ANSI L Spark ANSI L和CH +3 Spark ANSI Spark ANSI 和CH +4 Spark ANSI Spark ANSI CH +5 Spark ANSI Spark ANSI H +6 Spark ANSI Spark ANSI +Test with different input and replace values +_ _ +Spark SQL Spark SQL和CH +ANSI ANSI +Spark SQL Spark SQL和CH diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql index 4fd0791521d..4d0b5ecbe03 100644 --- a/tests/queries/0_stateless/03205_overlay.sql +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -7,7 +7,17 @@ SELECT overlay('hello', 456, 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT overlay('hello', 'world', 'two', 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT overlay('hello', 'world', 2, 'three'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT 'Positive test 1 with various combinations of const/non-const columns'; +SELECT 'Test with 3 arguments and various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', '_', 6), overlayUTF8('Spark SQL和CH', '_', 6); +SELECT overlay(materialize('Spark SQL'), '_', 6), overlayUTF8(materialize('Spark SQL和CH'), '_', 6); +SELECT overlay('Spark SQL', materialize('_'), 6), overlayUTF8('Spark SQL和CH', materialize('_'), 6); +SELECT overlay('Spark SQL', '_', materialize(6)), overlayUTF8('Spark SQL和CH', '_', materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), 6), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), 6); +SELECT overlay(materialize('Spark SQL'), '_', materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), '_', materialize(6)); +SELECT overlay('Spark SQL', materialize('_'), materialize(6)), overlayUTF8('Spark SQL和CH', materialize('_'), materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), materialize(6)); + +SELECT 'Test with 4 arguments and various combinations of const/non-const columns'; SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0); SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0); SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0); @@ -24,51 +34,15 @@ SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0) SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), materialize(0)); SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), materialize(0)); SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)); - -SELECT 'Positive test 2 with various combinations of const/non-const columns'; -SELECT overlay('Spark SQL', '_', 6), overlayUTF8('Spark SQL和CH', '_', 6); -SELECT overlay(materialize('Spark SQL'), '_', 6), overlayUTF8(materialize('Spark SQL和CH'), '_', 6); -SELECT overlay('Spark SQL', materialize('_'), 6), overlayUTF8('Spark SQL和CH', materialize('_'), 6); -SELECT overlay('Spark SQL', '_', materialize(6)), overlayUTF8('Spark SQL和CH', '_', materialize(6)); -SELECT overlay(materialize('Spark SQL'), materialize('_'), 6), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), 6); -SELECT overlay(materialize('Spark SQL'), '_', materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), '_', materialize(6)); -SELECT overlay('Spark SQL', materialize('_'), materialize(6)), overlayUTF8('Spark SQL和CH', materialize('_'), materialize(6)); -SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)), overlayUTF8(materialize('Spark SQL和CH'), materialize('_'), materialize(6)); - -SELECT 'Positive test 3 with various combinations of const/non-const columns'; -SELECT overlay('Spark SQL', 'CORE', 7), overlayUTF8('Spark SQL和CH', 'CORE', 7); -SELECT overlay(materialize('Spark SQL'), 'CORE', 7), overlayUTF8(materialize('Spark SQL和CH'), 'CORE', 7); -SELECT overlay('Spark SQL', materialize('CORE'), 7), overlayUTF8('Spark SQL和CH', materialize('CORE'), 7); -SELECT overlay('Spark SQL', 'CORE', materialize(7)), overlayUTF8('Spark SQL和CH', 'CORE', materialize(7)); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7), overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), 7); -SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)), overlayUTF8(materialize('Spark SQL和CH'), 'CORE', materialize(7)); -SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)), overlayUTF8('Spark SQL和CH', materialize('CORE'), materialize(7)); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)), overlayUTF8(materialize('Spark SQL和CH'), materialize('CORE'), materialize(7)); - -SELECT 'Positive test 4 with various combinations of const/non-const columns'; -SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, 0); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, 0); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, 0); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), 0); -SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, materialize(0)); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), 7, 0); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', materialize(7), 0); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), 'ANSI ', 7, materialize(0)); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), 0); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), 7, materialize(0)); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', 'ANSI ', materialize(7), materialize(0)); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)); - -SELECT 'Positive test 5 with various combinations of const/non-const columns'; -SELECT overlay('Spark SQL', 'tructured', 2, 4), overlayUTF8('Spark SQL和CH', 'tructured', 2, 4); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, 4); -SELECT overlay('Spark SQL', materialize('tructured'), 2, 4), overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, 4); -SELECT overlay('Spark SQL', 'tructured', materialize(2), 4), overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), 4); -SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)), overlayUTF8('Spark SQL和CH', 'tructured', 2, materialize(4)); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4), overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), 2, 4); -SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', materialize(2), 4); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)), overlayUTF8(materialize('Spark SQL和CH'), 'tructured', 2, materialize(4)); -SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4), overlayUTF8('Spark SQL和CH', materialize('tructured'), materialize(2), 4); -SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)), overlayUTF8('Spark SQL和CH', materialize('tructured'), 2, materialize(4)); -SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)), overlayUTF8('Spark SQL和CH', 'tructured', materialize(2), materialize(4)); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)), overlayUTF8(materialize('Spark SQL和CH'), materialize('tructured'), materialize(2), materialize(4)); + +SELECT 'Test with different offset values'; +WITH number - 12 as offset SELECT offset, overlay('Spark SQL', '_', offset), overlayUTF8('Spark SQL和CH', '_', offset) from numbers(26); + +SELECT 'Test with different length values'; +WITH number - 1 as length SELECT length, overlay('Spark SQL', 'ANSI ', 7, length), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, length) from numbers(8); + +SELECT 'Test with different input and replace values'; +SELECT overlay('', '_', 6), overlayUTF8('', '_', 6); +SELECT overlay('Spark SQL', '', 6), overlayUTF8('Spark SQL和CH', '', 6); +SELECT overlay('', 'ANSI ', 7, 0), overlayUTF8('', 'ANSI ', 7, 0); +SELECT overlay('Spark SQL', '', 7, 0), overlayUTF8('Spark SQL和CH', '', 7, 0); From c37fc005d39f880b6ab12b01e80954f162817790 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 19 Aug 2024 13:28:23 +0200 Subject: [PATCH 147/409] init --- src/Functions/FunctionSQLJSON.h | 48 ++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 18 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 83ed874c47b..66e0b4c2efd 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -123,7 +123,7 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, const ContextPtr & context) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, const bool & function_json_value_return_type_allow_complex) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); @@ -191,7 +191,7 @@ public: { /// Instead of creating a new generator for each row, we can reuse the same one. generator_json_path.reinitialize(); - added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context); + added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, function_json_value_return_type_allow_complex); } if (!added_to_column) { @@ -204,13 +204,25 @@ public: }; template typename Impl> -class FunctionSQLJSON : public IFunction, WithConstContext +class FunctionSQLJSON : public IFunction { public: static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - explicit FunctionSQLJSON(ContextPtr context_) : WithConstContext(context_) { } + explicit FunctionSQLJSON(ContextPtr context_) + { + max_parser_depth = context_->getSettingsRef().max_parser_depth; + max_parser_backtracks = context_->getSettingsRef().max_parser_backtracks; + allow_simdjson = context_->getSettingsRef().allow_simdjson; + function_json_value_return_type_allow_complex = context_->getSettingsRef().function_json_value_return_type_allow_complex; + function_json_value_return_type_allow_nullable = context_->getSettingsRef().function_json_value_return_type_allow_nullable; + } static constexpr auto name = Name::name; + bool max_parser_depth; + bool max_parser_backtracks; + bool allow_simdjson; + bool function_json_value_return_type_allow_complex; + bool function_json_value_return_type_allow_nullable; String getName() const override { return Name::name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -221,7 +233,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return Impl>::getReturnType( - Name::name, arguments, getContext()); + Name::name, arguments, function_json_value_return_type_allow_nullable); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -231,18 +243,18 @@ public: /// 2. Create ASTPtr /// 3. Parser(Tokens, ASTPtr) -> complete AST /// 4. Execute functions: call getNextItem on generator and handle each item - unsigned parse_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); - unsigned parse_backtracks = static_cast(getContext()->getSettingsRef().max_parser_backtracks); + unsigned parse_depth = static_cast(max_parser_depth); + unsigned parse_backtracks = static_cast(max_parser_backtracks); #if USE_SIMDJSON - if (getContext()->getSettingsRef().allow_simdjson) + if (allow_simdjson) return FunctionSQLJSONHelpers::Executor< Name, Impl>, - SimdJSONParser>::run(arguments, result_type, input_rows_count, parse_depth, parse_backtracks, getContext()); + SimdJSONParser>::run(arguments, result_type, input_rows_count, parse_depth, parse_backtracks, function_json_value_return_type_allow_complex); #endif return FunctionSQLJSONHelpers:: Executor>, DummyJSONParser>::run( - arguments, result_type, input_rows_count, parse_depth, parse_backtracks, getContext()); + arguments, result_type, input_rows_count, parse_depth, parse_backtracks, function_json_value_return_type_allow_complex); } }; @@ -267,11 +279,11 @@ class JSONExistsImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool &) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool &) { Element current_element = root; VisitorStatus status; @@ -305,9 +317,9 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr & context) + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool & function_json_value_return_type_allow_nullable) { - if (context->getSettingsRef().function_json_value_return_type_allow_nullable) + if (function_json_value_return_type_allow_nullable) { DataTypePtr string_type = std::make_shared(); return std::make_shared(string_type); @@ -320,7 +332,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr & context) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool & function_json_value_return_type_allow_complex) { Element current_element = root; VisitorStatus status; @@ -329,7 +341,7 @@ public: { if (status == VisitorStatus::Ok) { - if (context->getSettingsRef().function_json_value_return_type_allow_complex) + if (function_json_value_return_type_allow_complex) { break; } @@ -383,11 +395,11 @@ class JSONQueryImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const ContextPtr &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool &) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool &) { ColumnString & col_str = assert_cast(dest); From 6ee9cadc9550bf30c0c2e4b1176f803817c45bc0 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Mon, 19 Aug 2024 12:17:48 +0000 Subject: [PATCH 148/409] Set experimental_analyzer and no_parallel --- .../0_stateless/03222_datetime64_small_value_const.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql index 39266ba7992..a64ebd38ccf 100644 --- a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -1,5 +1,6 @@ -- Tags: shard set session_timezone = 'UTC'; -- don't randomize the session timezone +SET allow_experimental_analyzer = 1; select *, (select toDateTime64(0, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; select *, (select toDateTime64(5, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; @@ -38,3 +39,6 @@ select count(*) from distr_03222_dt64 where dt < (select toDateTime64('2004-07-2 drop table if exists shard_0.dt64_03222; drop table if exists shard_1.dt64_03222; drop table if exists distr_03222_dt64; + +drop database shard_0; +drop database shard_1; From 0d444e6708f04cf4a581f34246e57745552ab001 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 19 Aug 2024 15:39:24 +0200 Subject: [PATCH 149/409] review fixes --- src/Functions/FunctionSQLJSON.h | 35 +++++++++++++++++---------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 66e0b4c2efd..1c2a8d527f8 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -123,7 +123,7 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, const bool & function_json_value_return_type_allow_complex) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, const bool function_json_value_return_type_allow_complex) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); @@ -209,20 +209,15 @@ class FunctionSQLJSON : public IFunction public: static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } explicit FunctionSQLJSON(ContextPtr context_) + : max_parser_depth(context_->getSettingsRef().max_parser_depth), + max_parser_backtracks(context_->getSettingsRef().max_parser_backtracks), + allow_simdjson(context_->getSettingsRef().allow_simdjson), + function_json_value_return_type_allow_complex(context_->getSettingsRef().function_json_value_return_type_allow_complex), + function_json_value_return_type_allow_nullable(context_->getSettingsRef().function_json_value_return_type_allow_nullable) { - max_parser_depth = context_->getSettingsRef().max_parser_depth; - max_parser_backtracks = context_->getSettingsRef().max_parser_backtracks; - allow_simdjson = context_->getSettingsRef().allow_simdjson; - function_json_value_return_type_allow_complex = context_->getSettingsRef().function_json_value_return_type_allow_complex; - function_json_value_return_type_allow_nullable = context_->getSettingsRef().function_json_value_return_type_allow_nullable; } static constexpr auto name = Name::name; - bool max_parser_depth; - bool max_parser_backtracks; - bool allow_simdjson; - bool function_json_value_return_type_allow_complex; - bool function_json_value_return_type_allow_nullable; String getName() const override { return Name::name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -256,6 +251,12 @@ public: Executor>, DummyJSONParser>::run( arguments, result_type, input_rows_count, parse_depth, parse_backtracks, function_json_value_return_type_allow_complex); } +private: + const size_t max_parser_depth; + const size_t max_parser_backtracks; + const bool allow_simdjson; + const bool function_json_value_return_type_allow_complex; + const bool function_json_value_return_type_allow_nullable; }; struct NameJSONExists @@ -279,11 +280,11 @@ class JSONExistsImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool) { Element current_element = root; VisitorStatus status; @@ -317,7 +318,7 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool & function_json_value_return_type_allow_nullable) + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool function_json_value_return_type_allow_nullable) { if (function_json_value_return_type_allow_nullable) { @@ -332,7 +333,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool & function_json_value_return_type_allow_complex) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool function_json_value_return_type_allow_complex) { Element current_element = root; VisitorStatus status; @@ -395,11 +396,11 @@ class JSONQueryImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool &) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool) { ColumnString & col_str = assert_cast(dest); From 7e2804fad5e66b28074eb28fa1f6332c741cdad2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 19 Aug 2024 15:48:27 +0200 Subject: [PATCH 150/409] remove const (review) --- src/Functions/FunctionSQLJSON.h | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 1c2a8d527f8..4721f858f5c 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -123,7 +123,7 @@ public: class Executor { public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, const bool function_json_value_return_type_allow_complex) + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, uint32_t parse_depth, uint32_t parse_backtracks, bool function_json_value_return_type_allow_complex) { MutableColumnPtr to{result_type->createColumn()}; to->reserve(input_rows_count); @@ -280,11 +280,11 @@ class JSONExistsImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, bool) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, bool) { Element current_element = root; VisitorStatus status; @@ -318,7 +318,7 @@ class JSONValueImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool function_json_value_return_type_allow_nullable) + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, bool function_json_value_return_type_allow_nullable) { if (function_json_value_return_type_allow_nullable) { @@ -333,7 +333,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool function_json_value_return_type_allow_complex) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, bool function_json_value_return_type_allow_complex) { Element current_element = root; VisitorStatus status; @@ -396,11 +396,11 @@ class JSONQueryImpl public: using Element = typename JSONParser::Element; - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, const bool) { return std::make_shared(); } + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &, bool) { return std::make_shared(); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const bool) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, bool) { ColumnString & col_str = assert_cast(dest); From 527774d138fd2a8b077800c0897417cedb2113be Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 19 Aug 2024 15:26:17 +0000 Subject: [PATCH 151/409] 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 2b75221e1daf2935f40acbb0e593953073f981f7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 18:00:28 +0200 Subject: [PATCH 152/409] Reduce max_dynamic_paths in 01825_new_type_json_ghdata.sh to consume less memory --- tests/queries/0_stateless/01825_new_type_json_ghdata.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh index 33940caec29..f165223fb98 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON(max_dynamic_paths=100)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} \ --max_memory_usage 10G --query "INSERT INTO ghdata FORMAT JSONAsObject" From f9de579b41ac8b70c9638d658b37c77809e25324 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 18:01:44 +0200 Subject: [PATCH 153/409] Reduce max_dynamic_paths in another ghdata test --- .../0_stateless/01825_new_type_json_ghdata_insert_select.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh index 568ba2bd185..3f5fc91f8fc 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -10,9 +10,9 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_string" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_from_string" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON(max_dynamic_paths=100)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON(max_dynamic_paths=100)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} --max_memory_usage 10G -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_string FORMAT JSONAsString" From 31f236d4f256634a90a87408fb9b454cb2ced28b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 19 Aug 2024 17:28:27 +0000 Subject: [PATCH 154/409] Minor logging fixes --- programs/local/LocalServer.cpp | 6 +++--- programs/server/Server.cpp | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a8b774562f9..184f147a86a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -713,7 +713,7 @@ void LocalServer::processConfig() if (index_uncompressed_cache_size > max_cache_size) { index_uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(index_uncompressed_cache_size)); } global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio); @@ -723,7 +723,7 @@ void LocalServer::processConfig() if (index_mark_cache_size > max_cache_size) { index_mark_cache_size = max_cache_size; - LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(index_mark_cache_size)); } global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio); @@ -731,7 +731,7 @@ void LocalServer::processConfig() if (mmap_cache_size > max_cache_size) { mmap_cache_size = max_cache_size; - LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mmap_cache_size)); } global_context->setMMappedFileCache(mmap_cache_size); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 2c1f141bb1e..16fee378cf0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1420,7 +1420,7 @@ try if (index_uncompressed_cache_size > max_cache_size) { index_uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(index_uncompressed_cache_size)); } global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio); @@ -1430,7 +1430,7 @@ try if (index_mark_cache_size > max_cache_size) { index_mark_cache_size = max_cache_size; - LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(index_mark_cache_size)); } global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio); @@ -1438,7 +1438,7 @@ try if (mmap_cache_size > max_cache_size) { mmap_cache_size = max_cache_size; - LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mmap_cache_size)); } global_context->setMMappedFileCache(mmap_cache_size); @@ -1449,7 +1449,7 @@ try if (query_cache_max_size_in_bytes > max_cache_size) { query_cache_max_size_in_bytes = max_cache_size; - LOG_INFO(log, "Lowered query cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered query cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(query_cache_max_size_in_bytes)); } global_context->setQueryCache(query_cache_max_size_in_bytes, query_cache_max_entries, query_cache_query_cache_max_entry_size_in_bytes, query_cache_max_entry_size_in_rows); From 3ff97813f44d7bfb0561560d27164f4c680d5901 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 19 Aug 2024 17:28:29 +0000 Subject: [PATCH 155/409] add a test --- .../0_stateless/03227_test_sample_n.reference | 2 ++ .../0_stateless/03227_test_sample_n.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03227_test_sample_n.reference create mode 100644 tests/queries/0_stateless/03227_test_sample_n.sql diff --git a/tests/queries/0_stateless/03227_test_sample_n.reference b/tests/queries/0_stateless/03227_test_sample_n.reference new file mode 100644 index 00000000000..bc051d8d816 --- /dev/null +++ b/tests/queries/0_stateless/03227_test_sample_n.reference @@ -0,0 +1,2 @@ +50000 +1 diff --git a/tests/queries/0_stateless/03227_test_sample_n.sql b/tests/queries/0_stateless/03227_test_sample_n.sql new file mode 100644 index 00000000000..d38bdd0a3ee --- /dev/null +++ b/tests/queries/0_stateless/03227_test_sample_n.sql @@ -0,0 +1,20 @@ +CREATE TABLE IF NOT EXISTS table_name +( +id UInt64 +) +ENGINE = MergeTree() +ORDER BY cityHash64(id) +SAMPLE BY cityHash64(id); + +INSERT INTO table_name SELECT rand() from system.numbers limit 10000; +INSERT INTO table_name SELECT rand() from system.numbers limit 10000; +INSERT INTO table_name SELECT rand() from system.numbers limit 10000; +INSERT INTO table_name SELECT rand() from system.numbers limit 10000; +INSERT INTO table_name SELECT rand() from system.numbers limit 10000; + +select count() from table_name; +SELECT count() < 50 * 5 FROM ( + SELECT * FROM table_name SAMPLE 50 +); + +DROP TABLE table_name; From b3c3ea45ebdd6fae840f7ce5b47f2ce0226ce7fb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 19 Aug 2024 14:24:58 +0000 Subject: [PATCH 156/409] Slightly better logging --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 27bfcbbddcf..4c0da28c3c4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -190,6 +190,8 @@ MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) const { + LOG_TRACE(logger, "Start writing vector similarity index"); + if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name)); @@ -207,6 +209,8 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) { + LOG_TRACE(logger, "Start loading vector similarity index"); + UInt64 file_version; readIntBinary(file_version, istr); if (file_version != FILE_FORMAT_VERSION) From 2f5fb44695fdb1f66a927f1402512dbd065b8c41 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 19:35:44 +0200 Subject: [PATCH 157/409] Better check and error code --- src/DataTypes/DataTypeObject.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 11fffd8769b..9846ae876ca 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -56,8 +56,8 @@ DataTypeObject::DataTypeObject( /// Don't log errors to stderr. options.set_log_errors(false); auto regexp = re2::RE2(regexp_str, options); - if (!regexp.error().empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid regexp '{}': {}", regexp_str, regexp.error()); + if (!regexp.ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Invalid regexp '{}': {}", regexp_str, regexp.error()); } for (const auto & [typed_path, type] : typed_paths) From f49fe765328c3637ab781a986a0f6c463fc0ecde Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 19:36:00 +0200 Subject: [PATCH 158/409] Add missing error code --- src/DataTypes/DataTypeObject.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 9846ae876ca..0d410d6f24c 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes { extern const int UNEXPECTED_AST_STRUCTURE; extern const int BAD_ARGUMENTS; + extern const int CANNOT_COMPILE_REGEXP; } DataTypeObject::DataTypeObject( From 51c5b91b0a58f6577adba0c1e5717f649104c17b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 19:36:20 +0200 Subject: [PATCH 159/409] Update error code in tests --- tests/queries/0_stateless/03227_json_invalid_regexp.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03227_json_invalid_regexp.sql b/tests/queries/0_stateless/03227_json_invalid_regexp.sql index 734dea1aac6..d98e2ade29d 100644 --- a/tests/queries/0_stateless/03227_json_invalid_regexp.sql +++ b/tests/queries/0_stateless/03227_json_invalid_regexp.sql @@ -1,4 +1,4 @@ set allow_experimental_json_type = 1; -create table test (json JSON(SKIP REGEXP '[]')) engine=Memory(); -- {serverError BAD_ARGUMENTS} -create table test (json JSON(SKIP REGEXP '+')) engine=Memory(); -- {serverError BAD_ARGUMENTS}; +create table test (json JSON(SKIP REGEXP '[]')) engine=Memory(); -- {serverError CANNOT_COMPILE_REGEXP} +create table test (json JSON(SKIP REGEXP '+')) engine=Memory(); -- {serverError CANNOT_COMPILE_REGEXP}; From 2d998a9f6a12499dcc82186ff9f0c5edaf173c6c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 20:19:11 +0200 Subject: [PATCH 160/409] Fix test --- .../03037_dynamic_merges_1_horizontal_compact_merge_tree.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index 46f1c78b255..5e7d1ee9826 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -33,5 +33,3 @@ optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); drop table test; - -select 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 161/409] 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 bc9cac605a58b4dce38bf06da889639f5bd62438 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 19 Aug 2024 22:00:35 +0000 Subject: [PATCH 162/409] Fix enumerating dynamic subcolumns --- src/DataTypes/IDataType.cpp | 1 + src/DataTypes/Serializations/ISerialization.h | 1 + .../Serializations/SerializationDynamic.cpp | 2 +- .../Serializations/SerializationObject.cpp | 2 +- ...dynamic_subcolumns_enumerate_streams.reference | 15 +++++++++++++++ ...03227_dynamic_subcolumns_enumerate_streams.sql | 9 +++++++++ 6 files changed, 28 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.reference create mode 100644 tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.sql diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 7fd8a85aeca..d14461a4ff1 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -150,6 +150,7 @@ std::unique_ptr IDataType::getSubcolumnData( ISerialization::EnumerateStreamsSettings settings; settings.position_independent_encoding = false; + settings.enumerate_dynamic_streams = false; data.serialization->enumerateStreams(settings, callback_with_data, data); if (!res && data.type->hasDynamicSubcolumnsData()) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 480d5a4f7c4..d19cb0dd365 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -241,6 +241,7 @@ public: { SubstreamPath path; bool position_independent_encoding = true; + bool enumerate_dynamic_streams = true; }; virtual void enumerateStreams( diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 6bba87c40fa..10a80aa8a0d 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -64,7 +64,7 @@ void SerializationDynamic::enumerateStreams( const auto * deserialize_state = data.deserialize_state ? checkAndGetState(data.deserialize_state) : nullptr; /// If column is nullptr and we don't have deserialize state yet, nothing to enumerate as we don't have any variants. - if (!column_dynamic && !deserialize_state) + if (!settings.enumerate_dynamic_streams || (!column_dynamic && !deserialize_state)) return; const auto & variant_type = column_dynamic ? column_dynamic->getVariantInfo().variant_type : checkAndGetState(deserialize_state->structure_state)->variant_type; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 2dd25e540cc..45121b942e6 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -130,7 +130,7 @@ void SerializationObject::enumerateStreams(EnumerateStreamsSettings & settings, } /// If column or deserialization state was provided, iterate over dynamic paths, - if (column_object || structure_state) + if (settings.enumerate_dynamic_streams && (column_object || structure_state)) { /// Enumerate dynamic paths in sorted order for consistency. const auto * dynamic_paths = column_object ? &column_object->getDynamicPaths() : nullptr; diff --git a/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.reference b/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.reference new file mode 100644 index 00000000000..b9d711bf9f0 --- /dev/null +++ b/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.reference @@ -0,0 +1,15 @@ +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +str_3 +str_4 +\N +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.sql b/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.sql new file mode 100644 index 00000000000..e451521faca --- /dev/null +++ b/tests/queries/0_stateless/03227_dynamic_subcolumns_enumerate_streams.sql @@ -0,0 +1,9 @@ +set allow_experimental_json_type=1; +drop table if exists test; +create table test (json JSON) engine=Memory; +insert into test select toJSONString(map('a', 'str_' || number)) from numbers(5); +select json.a.String from test; +select json.a.:String from test; +select json.a.UInt64 from test; +drop table test; + From 902e7b6f29fe90007525f1f74adbf26e72b90bde Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 19 Aug 2024 23:58:48 +0000 Subject: [PATCH 163/409] 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 246b050e300da89c408b32eba70b5e1326e28764 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 20 Aug 2024 00:38:50 +0000 Subject: [PATCH 164/409] Fix filename --- ...er_pushdown_bug.sh.sh => 02841_parquet_filter_pushdown_bug.sh} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02841_parquet_filter_pushdown_bug.sh.sh => 02841_parquet_filter_pushdown_bug.sh} (100%) diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh similarity index 100% rename from tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh rename to tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh From 9b55180184c0f38cf88bf5902687603cf903ee40 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 9 Aug 2024 14:22:33 +0000 Subject: [PATCH 165/409] CI: Move out scripts from dockers --- docker/images.json | 7 +---- docker/test/fasttest/Dockerfile | 3 -- docker/test/stateful/Dockerfile | 4 --- docker/test/stateful/setup_minio.sh | 1 - docker/test/stateless/Dockerfile | 12 -------- docker/test/stress/Dockerfile | 3 -- docker/test/upgrade/Dockerfile | 29 ------------------- docker/test/util/Dockerfile | 2 -- tests/ci/.mypy.ini | 3 +- tests/ci/ci_config.py | 5 +++- tests/ci/ci_definitions.py | 7 +++-- tests/ci/docker_images_check.py | 2 +- tests/ci/fast_test_check.py | 7 ++--- tests/ci/functional_test_check.py | 20 ++++++------- tests/ci/stress_check.py | 17 +++++++---- tests/ci/upgrade_check.py | 2 +- .../docker_scripts}/attach_gdb.lib | 2 +- .../docker_scripts}/create.sql | 0 .../docker_scripts/fasttest_runner.sh | 2 +- .../process_functional_tests_result.py | 24 +++++++-------- .../docker_scripts}/setup_hdfs_minicluster.sh | 2 +- .../docker_scripts}/setup_minio.sh | 2 +- .../docker_scripts/stateful_runner.sh | 13 +++++---- .../docker_scripts/stateless_runner.sh | 22 +++++++++----- .../docker_scripts/stress_runner.sh | 21 +++++++------- .../docker_scripts}/stress_tests.lib | 2 +- .../docker_scripts/upgrade_runner.sh | 14 ++++----- .../docker_scripts}/utils.lib | 0 28 files changed, 94 insertions(+), 134 deletions(-) delete mode 120000 docker/test/stateful/setup_minio.sh delete mode 100644 docker/test/upgrade/Dockerfile rename {docker/test/stateless => tests/docker_scripts}/attach_gdb.lib (98%) rename {docker/test/stateful => tests/docker_scripts}/create.sql (100%) rename docker/test/fasttest/run.sh => tests/docker_scripts/fasttest_runner.sh (99%) rename {docker/test/util => tests/docker_scripts}/process_functional_tests_result.py (92%) rename {docker/test/stateless => tests/docker_scripts}/setup_hdfs_minicluster.sh (95%) rename {docker/test/stateless => tests/docker_scripts}/setup_minio.sh (98%) rename docker/test/stateful/run.sh => tests/docker_scripts/stateful_runner.sh (98%) rename docker/test/stateless/run.sh => tests/docker_scripts/stateless_runner.sh (97%) rename docker/test/stress/run.sh => tests/docker_scripts/stress_runner.sh (97%) mode change 100644 => 100755 rename {docker/test/stateless => tests/docker_scripts}/stress_tests.lib (99%) rename docker/test/upgrade/run.sh => tests/docker_scripts/upgrade_runner.sh (96%) mode change 100644 => 100755 rename {docker/test/stateless => tests/docker_scripts}/utils.lib (100%) diff --git a/docker/images.json b/docker/images.json index 716b76ee217..055394b69e6 100644 --- a/docker/images.json +++ b/docker/images.json @@ -47,8 +47,7 @@ "docker/test/stateful": { "name": "clickhouse/stateful-test", "dependent": [ - "docker/test/stress", - "docker/test/upgrade" + "docker/test/stress" ] }, "docker/test/unit": { @@ -59,10 +58,6 @@ "name": "clickhouse/stress-test", "dependent": [] }, - "docker/test/upgrade": { - "name": "clickhouse/upgrade-check", - "dependent": [] - }, "docker/test/integration/runner": { "name": "clickhouse/integration-tests-runner", "dependent": [] diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 5d311c673a4..ca93b24f66e 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -93,6 +93,3 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV COMMIT_SHA='' ENV PULL_REQUEST_NUMBER='' ENV COPY_CLICKHOUSE_BINARY_TO_OUTPUT=0 - -COPY run.sh / -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 0daf88cad7e..9aa936cb069 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -10,7 +10,3 @@ RUN apt-get update -y \ npm \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -COPY create.sql / -COPY run.sh / -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateful/setup_minio.sh b/docker/test/stateful/setup_minio.sh deleted file mode 120000 index 0d539f72cb3..00000000000 --- a/docker/test/stateful/setup_minio.sh +++ /dev/null @@ -1 +0,0 @@ -../stateless/setup_minio.sh \ No newline at end of file diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index b0c4914a4e8..69f81b35a95 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -85,18 +85,6 @@ RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoo ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -ENV CLICKHOUSE_GRPC_CLIENT="/usr/share/clickhouse-utils/grpc-client/clickhouse-grpc-client.py" RUN npm install -g azurite@3.30.0 \ && npm install -g tslib && npm install -g node - -COPY run.sh / -COPY setup_minio.sh / -COPY setup_hdfs_minicluster.sh / -COPY attach_gdb.lib / -COPY utils.lib / - -# We store stress_tests.lib in stateless image to avoid duplication of this file in stress and upgrade tests -COPY stress_tests.lib / - -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 0f81a1cd07f..ecb98a4e3ed 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -22,8 +22,5 @@ RUN apt-get update -y \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -COPY run.sh / - ENV EXPORT_S3_STORAGE_POLICIES=1 -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/upgrade/Dockerfile b/docker/test/upgrade/Dockerfile deleted file mode 100644 index 78d912fd031..00000000000 --- a/docker/test/upgrade/Dockerfile +++ /dev/null @@ -1,29 +0,0 @@ -# rebuild in #33610 -# docker build -t clickhouse/upgrade-check . -ARG FROM_TAG=latest -FROM clickhouse/stateful-test:$FROM_TAG - -RUN apt-get update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get install --yes --no-install-recommends \ - bash \ - tzdata \ - parallel \ - expect \ - python3 \ - python3-lxml \ - python3-termcolor \ - python3-requests \ - curl \ - sudo \ - openssl \ - netcat-openbsd \ - brotli \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -COPY run.sh / - -ENV EXPORT_S3_STORAGE_POLICIES=1 - -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index 8b949ed95db..6b9fb94a4c6 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -56,7 +56,5 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -COPY process_functional_tests_result.py / - COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb ENV PATH="/opt/gdb/bin:${PATH}" diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini index f12d27979ce..ecb4aef87dd 100644 --- a/tests/ci/.mypy.ini +++ b/tests/ci/.mypy.ini @@ -15,4 +15,5 @@ warn_return_any = True no_implicit_reexport = True strict_equality = True extra_checks = True -ignore_missing_imports = True \ No newline at end of file +ignore_missing_imports = True +logging-fstring-interpolation = False \ No newline at end of file diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5453bffd9c6..58de25f039f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -535,7 +535,10 @@ class CI: JobNames.FAST_TEST: JobConfig( pr_only=True, digest=DigestConfig( - include_paths=["./tests/queries/0_stateless/"], + include_paths=[ + "./tests/queries/0_stateless/", + "./tests/docker_scripts/", + ], exclude_files=[".md"], docker=["clickhouse/fasttest"], ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 13c222b10b9..1cdb3f1487e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -415,6 +415,7 @@ class CommonJobConfigs: "./tests/clickhouse-test", "./tests/config", "./tests/*.txt", + "./tests/docker_scripts/", ], exclude_files=[".md"], docker=["clickhouse/stateless-test"], @@ -431,6 +432,7 @@ class CommonJobConfigs: "./tests/clickhouse-test", "./tests/config", "./tests/*.txt", + "./tests/docker_scripts/", ], exclude_files=[".md"], docker=["clickhouse/stateful-test"], @@ -448,6 +450,7 @@ class CommonJobConfigs: "./tests/clickhouse-test", "./tests/config", "./tests/*.txt", + "./tests/docker_scripts/", ], exclude_files=[".md"], docker=["clickhouse/stress-test"], @@ -459,9 +462,9 @@ class CommonJobConfigs: UPGRADE_TEST = JobConfig( job_name_keyword="upgrade", digest=DigestConfig( - include_paths=["./tests/ci/upgrade_check.py"], + include_paths=["./tests/ci/upgrade_check.py", "./tests/docker_scripts/"], exclude_files=[".md"], - docker=["clickhouse/upgrade-check"], + docker=["clickhouse/stress-test"], ), run_command="upgrade_check.py", runner_type=Runners.STRESS_TESTER, diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 786a529e0a9..c8dbcd10245 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -93,7 +93,7 @@ def process_single_image( results = [] # type: TestResults for ver in versions: stopwatch = Stopwatch() - for i in range(5): + for i in range(2): success, build_log = build_and_push_one_image( image, ver, additional_cache, push, from_tag ) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index ed727dd3659..55eefcf9714 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -31,15 +31,14 @@ def get_fasttest_cmd( "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls "--network=host " # required to get access to IAM credentials f"-e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output " - f"-e FASTTEST_SOURCE=/ClickHouse " + f"-e FASTTEST_SOURCE=/repo " f"-e FASTTEST_CMAKE_FLAGS='-DCOMPILER_CACHE=sccache' " f"-e PULL_REQUEST_NUMBER={pr_number} -e COMMIT_SHA={commit_sha} " f"-e COPY_CLICKHOUSE_BINARY_TO_OUTPUT=1 " f"-e SCCACHE_BUCKET={S3_BUILDS_BUCKET} -e SCCACHE_S3_KEY_PREFIX=ccache/sccache " "-e stage=clone_submodules " - f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/ClickHouse " - f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " - f"--volume={output_path}:/test_output {image}" + f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/repo " + f"--volume={output_path}:/test_output {image} /repo/tests/docker_scripts/fasttest_runner.sh" ) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index d08f98fa05f..ce2ead59d1a 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -119,24 +119,24 @@ def get_run_command( envs += [f"-e {e}" for e in additional_envs] env_str = " ".join(envs) - volume_with_broken_test = ( - f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " - if "analyzer" not in check_name - else "" - ) + + if "stateful" in check_name.lower(): + run_script = "/repo/tests/docker_scripts/stateful_runner.sh" + elif "stateless" in check_name.lower(): + run_script = "/repo/tests/docker_scripts/stateless_runner.sh" + else: + assert False return ( f"docker run --rm --name func-tester --volume={builds_path}:/package_folder " # For dmesg and sysctl "--privileged " - f"{ci_logs_args}" - f"--volume={repo_path}/tests:/usr/share/clickhouse-test " - f"--volume={repo_path}/utils/grpc-client:/usr/share/clickhouse-utils/grpc-client " - f"{volume_with_broken_test}" + f"{ci_logs_args} " + f"--volume={repo_path}:/repo " f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" + f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image} {run_script}" ) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 85da601e379..f9656e60448 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -57,10 +57,16 @@ def get_run_command( additional_envs: List[str], ci_logs_args: str, image: DockerImage, + upgrade_check: bool, ) -> str: envs = [f"-e {e}" for e in additional_envs] env_str = " ".join(envs) + if upgrade_check: + run_script = "/repo/tests/docker_scripts/upgrade_runner.sh" + else: + run_script = "/repo/tests/docker_scripts/stress_runner.sh" + cmd = ( "docker run --cap-add=SYS_PTRACE " # For dmesg and sysctl @@ -70,8 +76,8 @@ def get_run_command( f"{ci_logs_args}" f"--volume={build_path}:/package_folder " f"--volume={result_path}:/test_output " - f"--volume={repo_tests_path}:/usr/share/clickhouse-test " - f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} " + f"--volume={repo_tests_path}/..:/repo " + f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} {run_script}" ) return cmd @@ -128,7 +134,7 @@ def process_results( return state, description, test_results, additional_files -def run_stress_test(docker_image_name: str) -> None: +def run_stress_test(upgrade_check: bool = False) -> None: logging.basicConfig(level=logging.INFO) for handler in logging.root.handlers: # pylint: disable=protected-access @@ -148,7 +154,7 @@ def run_stress_test(docker_image_name: str) -> None: pr_info = PRInfo() - docker_image = pull_image(get_docker_image(docker_image_name)) + docker_image = pull_image(get_docker_image("clickhouse/stress-test")) packages_path = temp_path / "packages" packages_path.mkdir(parents=True, exist_ok=True) @@ -177,6 +183,7 @@ def run_stress_test(docker_image_name: str) -> None: additional_envs, ci_logs_args, docker_image, + upgrade_check, ) logging.info("Going to run stress test: %s", run_command) @@ -208,4 +215,4 @@ def run_stress_test(docker_image_name: str) -> None: if __name__ == "__main__": - run_stress_test("clickhouse/stress-test") + run_stress_test() diff --git a/tests/ci/upgrade_check.py b/tests/ci/upgrade_check.py index 83b6f9e299f..8662611dffe 100644 --- a/tests/ci/upgrade_check.py +++ b/tests/ci/upgrade_check.py @@ -1,4 +1,4 @@ import stress_check if __name__ == "__main__": - stress_check.run_stress_test("clickhouse/upgrade-check") + stress_check.run_stress_test(upgrade_check=True) diff --git a/docker/test/stateless/attach_gdb.lib b/tests/docker_scripts/attach_gdb.lib similarity index 98% rename from docker/test/stateless/attach_gdb.lib rename to tests/docker_scripts/attach_gdb.lib index 2f1375a2f0f..4170a19176c 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/tests/docker_scripts/attach_gdb.lib @@ -1,7 +1,7 @@ #!/bin/bash # shellcheck source=./utils.lib -source /utils.lib +source /repo/tests/docker_scripts/utils.lib function attach_gdb_to_clickhouse() { diff --git a/docker/test/stateful/create.sql b/tests/docker_scripts/create.sql similarity index 100% rename from docker/test/stateful/create.sql rename to tests/docker_scripts/create.sql diff --git a/docker/test/fasttest/run.sh b/tests/docker_scripts/fasttest_runner.sh similarity index 99% rename from docker/test/fasttest/run.sh rename to tests/docker_scripts/fasttest_runner.sh index 9920326b11c..1eaba2c7cdf 100755 --- a/docker/test/fasttest/run.sh +++ b/tests/docker_scripts/fasttest_runner.sh @@ -325,7 +325,7 @@ case "$stage" in ;& "run_tests") run_tests ||: - /process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \ + /repo/tests/docker_scripts/process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \ --out-results-file "$FASTTEST_OUTPUT/test_results.tsv" \ --out-status-file "$FASTTEST_OUTPUT/check_status.tsv" || echo -e "failure\tCannot parse results" > "$FASTTEST_OUTPUT/check_status.tsv" ;; diff --git a/docker/test/util/process_functional_tests_result.py b/tests/docker_scripts/process_functional_tests_result.py similarity index 92% rename from docker/test/util/process_functional_tests_result.py rename to tests/docker_scripts/process_functional_tests_result.py index ec9e14b1430..1dc3090484c 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/tests/docker_scripts/process_functional_tests_result.py @@ -32,7 +32,7 @@ def process_test_log(log_path, broken_tests): success_finish = False test_results = [] test_end = True - with open(log_path, "r") as test_file: + with open(log_path, "r", encoding="utf-8") as test_file: for line in test_file: original_line = line line = line.strip() @@ -150,7 +150,7 @@ def process_result(result_path, broken_tests): if result_path and os.path.exists(result_path): ( - total, + _total, skipped, unknown, failed, @@ -191,11 +191,11 @@ def process_result(result_path, broken_tests): else: description = "" - description += "fail: {}, passed: {}".format(failed, success) + description += f"fail: {failed}, passed: {success}" if skipped != 0: - description += ", skipped: {}".format(skipped) + description += f", skipped: {skipped}" if unknown != 0: - description += ", unknown: {}".format(unknown) + description += f", unknown: {unknown}" else: state = "failure" description = "Output log doesn't exist" @@ -205,10 +205,10 @@ def process_result(result_path, broken_tests): def write_results(results_file, status_file, results, status): - with open(results_file, "w") as f: + with open(results_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerows(results) - with open(status_file, "w") as f: + with open(status_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerow(status) @@ -221,15 +221,15 @@ if __name__ == "__main__": parser.add_argument("--in-results-dir", default="/test_output/") parser.add_argument("--out-results-file", default="/test_output/test_results.tsv") parser.add_argument("--out-status-file", default="/test_output/check_status.tsv") - parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt") + parser.add_argument("--broken-tests", default="/repo/tests/analyzer_tech_debt.txt") args = parser.parse_args() - broken_tests = list() + broken_tests = [] if os.path.exists(args.broken_tests): - logging.info(f"File {args.broken_tests} with broken tests found") - with open(args.broken_tests) as f: + print(f"File {args.broken_tests} with broken tests found") + with open(args.broken_tests, encoding="utf-8") as f: broken_tests = f.read().splitlines() - logging.info(f"Broken tests in the list: {len(broken_tests)}") + print(f"Broken tests in the list: {len(broken_tests)}") state, description, test_results = process_result(args.in_results_dir, broken_tests) logging.info("Result parsed") diff --git a/docker/test/stateless/setup_hdfs_minicluster.sh b/tests/docker_scripts/setup_hdfs_minicluster.sh similarity index 95% rename from docker/test/stateless/setup_hdfs_minicluster.sh rename to tests/docker_scripts/setup_hdfs_minicluster.sh index 15a54f59096..622270ba5d5 100755 --- a/docker/test/stateless/setup_hdfs_minicluster.sh +++ b/tests/docker_scripts/setup_hdfs_minicluster.sh @@ -5,7 +5,7 @@ set -e -x -a -u ls -lha -cd hadoop-3.3.1 +cd /hadoop-3.3.1 export JAVA_HOME=/usr mkdir -p target/test/data diff --git a/docker/test/stateless/setup_minio.sh b/tests/docker_scripts/setup_minio.sh similarity index 98% rename from docker/test/stateless/setup_minio.sh rename to tests/docker_scripts/setup_minio.sh index c1508df7e82..40e93e713a1 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/tests/docker_scripts/setup_minio.sh @@ -143,7 +143,7 @@ main() { fi start_minio setup_minio "$1" - upload_data "${query_dir}" "${2:-/usr/share/clickhouse-test}" + upload_data "${query_dir}" "${2:-/repo/tests/}" setup_aws_credentials } diff --git a/docker/test/stateful/run.sh b/tests/docker_scripts/stateful_runner.sh similarity index 98% rename from docker/test/stateful/run.sh rename to tests/docker_scripts/stateful_runner.sh index a489a4b6e14..86f6a299ad3 100755 --- a/docker/test/stateful/run.sh +++ b/tests/docker_scripts/stateful_runner.sh @@ -14,17 +14,17 @@ dpkg -i package_folder/clickhouse-common-static-dbg_*.deb dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb -ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test # shellcheck disable=SC1091 -source /utils.lib +source /repo/tests/docker_scripts/utils.lib # install test configs -/usr/share/clickhouse-test/config/install.sh +/repo/tests/config/install.sh azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence & -./setup_minio.sh stateful +/repo/tests/docker_scripts/setup_minio.sh stateful ./mc admin trace clickminio > /test_output/minio.log & MC_ADMIN_PID=$! @@ -105,7 +105,7 @@ setup_logs_replication clickhouse-client --query "SHOW DATABASES" clickhouse-client --query "CREATE DATABASE datasets" -clickhouse-client --multiquery < create.sql +clickhouse-client --multiquery < /repo/tests/docker_scripts/create.sql clickhouse-client --query "SHOW TABLES FROM datasets" if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then @@ -237,6 +237,7 @@ function run_tests() --hung-check --print-time --capture-client-stacktrace + --queries "/repo/tests/queries" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" ) @@ -259,7 +260,7 @@ ls -la ./ echo "Files in root directory" ls -la / -/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv +/repo/tests/docker_scripts/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv sudo clickhouse stop ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then diff --git a/docker/test/stateless/run.sh b/tests/docker_scripts/stateless_runner.sh similarity index 97% rename from docker/test/stateless/run.sh rename to tests/docker_scripts/stateless_runner.sh index 6973eebbac3..671b1f5ca71 100755 --- a/docker/test/stateless/run.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -1,10 +1,13 @@ #!/bin/bash +# fail on errors, verbose and export all env variables +set -e -x -a + # shellcheck disable=SC1091 source /setup_export_logs.sh # shellcheck source=../stateless/stress_tests.lib -source /stress_tests.lib +source /repo/tests/docker_scripts/stress_tests.lib # Avoid overlaps with previous runs dmesg --clear @@ -39,20 +42,22 @@ if [[ -z "$BUGFIX_VALIDATE_CHECK" ]]; then chc --version || exit 1 fi -ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -sf /repo/tests/clickhouse-test /usr/bin/clickhouse-test + +export CLICKHOUSE_GRPC_CLIENT="/repo/utils/grpc-client/clickhouse-grpc-client.py" # shellcheck disable=SC1091 -source /attach_gdb.lib +source /repo/tests/docker_scripts/attach_gdb.lib # shellcheck disable=SC1091 -source /utils.lib +source /repo/tests/docker_scripts/utils.lib # install test configs -/usr/share/clickhouse-test/config/install.sh +/repo/tests/config/install.sh -./setup_minio.sh stateless +/repo/tests/docker_scripts/setup_minio.sh stateless -./setup_hdfs_minicluster.sh +/repo/tests/docker_scripts/setup_hdfs_minicluster.sh config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml @@ -316,6 +321,7 @@ function run_tests() --print-time --no-drop-if-fail --capture-client-stacktrace + --queries "/repo/tests/queries" --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" ) @@ -341,7 +347,7 @@ ls -la ./ echo "Files in root directory" ls -la / -/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv +/repo/tests/docker_scripts/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv clickhouse-client -q "system flush logs" ||: diff --git a/docker/test/stress/run.sh b/tests/docker_scripts/stress_runner.sh old mode 100644 new mode 100755 similarity index 97% rename from docker/test/stress/run.sh rename to tests/docker_scripts/stress_runner.sh index b21114e456f..7666398e10b --- a/docker/test/stress/run.sh +++ b/tests/docker_scripts/stress_runner.sh @@ -3,26 +3,25 @@ # shellcheck disable=SC2086 # shellcheck disable=SC2024 +set -x + # Avoid overlaps with previous runs dmesg --clear # shellcheck disable=SC1091 source /setup_export_logs.sh -set -x - -# we mount tests folder from repo to /usr/share -ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress -ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/clickhouse-test/ci/stress.py /usr/bin/stress +ln -s /repo/tests/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib # shellcheck source=../stateless/attach_gdb.lib -source /attach_gdb.lib +source /repo/tests/docker_scripts/attach_gdb.lib # shellcheck source=../stateless/stress_tests.lib -source /stress_tests.lib +source /repo/tests/docker_scripts/stress_tests.lib # shellcheck disable=SC1091 -source /utils.lib +source /repo/tests/docker_scripts/utils.lib install_packages package_folder @@ -55,7 +54,7 @@ export ZOOKEEPER_FAULT_INJECTION=1 # available for dump via clickhouse-local configure -./setup_minio.sh stateless # to have a proper environment +/repo/tests/docker_scripts/setup_minio.sh stateless # to have a proper environment config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml @@ -64,7 +63,7 @@ start_server setup_logs_replication clickhouse-client --query "CREATE DATABASE datasets" -clickhouse-client --multiquery < create.sql +clickhouse-client --multiquery < /repo/tests/docker_scripts/create.sql clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" @@ -267,7 +266,7 @@ fi start_server -stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ +python3 /repo/tests/ci/stress.py --hung-check --drop-databases --output-folder /test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ && echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \ || echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv diff --git a/docker/test/stateless/stress_tests.lib b/tests/docker_scripts/stress_tests.lib similarity index 99% rename from docker/test/stateless/stress_tests.lib rename to tests/docker_scripts/stress_tests.lib index 51aa299f7a6..e2b5d983fcb 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -42,7 +42,7 @@ function configure() # install test configs export USE_DATABASE_ORDINARY=1 export EXPORT_S3_STORAGE_POLICIES=1 - /usr/share/clickhouse-test/config/install.sh + /repo/tests/config/install.sh # avoid too slow startup sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ diff --git a/docker/test/upgrade/run.sh b/tests/docker_scripts/upgrade_runner.sh old mode 100644 new mode 100755 similarity index 96% rename from docker/test/upgrade/run.sh rename to tests/docker_scripts/upgrade_runner.sh index e9768873f6a..ece75ebf782 --- a/docker/test/upgrade/run.sh +++ b/tests/docker_scripts/upgrade_runner.sh @@ -9,20 +9,20 @@ dmesg --clear set -x # we mount tests folder from repo to /usr/share -ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress -ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test -ln -s /usr/share/clickhouse-test/ci/download_release_packages.py /usr/bin/download_release_packages -ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag +ln -s /repo/tests/ci/stress.py /usr/bin/stress +ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/ci/download_release_packages.py /usr/bin/download_release_packages +ln -s /repo/tests/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib # shellcheck source=../stateless/attach_gdb.lib -source /attach_gdb.lib +source /repo/tests/docker_scripts/attach_gdb.lib # shellcheck source=../stateless/stress_tests.lib -source /stress_tests.lib +source /repo/tests/docker_scripts/stress_tests.lib azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & -./setup_minio.sh stateless # to have a proper environment +/repo/tests/docker_scripts/setup_minio.sh stateless # to have a proper environment echo "Get previous release tag" # shellcheck disable=SC2016 diff --git a/docker/test/stateless/utils.lib b/tests/docker_scripts/utils.lib similarity index 100% rename from docker/test/stateless/utils.lib rename to tests/docker_scripts/utils.lib From 67f1dc276f0af5657eebf2153ffca3be36462157 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 20 Aug 2024 08:37:33 +0000 Subject: [PATCH 166/409] Update version_date.tsv and changelogs after v23.8.16.16-lts --- docs/changelogs/v23.8.16.16-lts.md | 38 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 39 insertions(+) create mode 100644 docs/changelogs/v23.8.16.16-lts.md diff --git a/docs/changelogs/v23.8.16.16-lts.md b/docs/changelogs/v23.8.16.16-lts.md new file mode 100644 index 00000000000..9532db4fb0a --- /dev/null +++ b/docs/changelogs/v23.8.16.16-lts.md @@ -0,0 +1,38 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.16.16-lts (b80cac57ead) FIXME as compared to v23.8.15.35-lts (060ff8e813a) + +#### Improvement +* Backported in [#66962](https://github.com/ClickHouse/ClickHouse/issues/66962): Added support for parameterized view with analyzer to not analyze create parameterized view. Refactor existing parameterized view logic to not analyze create parameterized view. [#54211](https://github.com/ClickHouse/ClickHouse/pull/54211) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Backported in [#65461](https://github.com/ClickHouse/ClickHouse/issues/65461): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Backported in [#65880](https://github.com/ClickHouse/ClickHouse/issues/65880): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65912](https://github.com/ClickHouse/ClickHouse/issues/65912): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65351](https://github.com/ClickHouse/ClickHouse/issues/65351): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66037](https://github.com/ClickHouse/ClickHouse/issues/66037): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65281](https://github.com/ClickHouse/ClickHouse/issues/65281): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65368](https://github.com/ClickHouse/ClickHouse/issues/65368): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#65782](https://github.com/ClickHouse/ClickHouse/issues/65782): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65743](https://github.com/ClickHouse/ClickHouse/issues/65743): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65926](https://github.com/ClickHouse/ClickHouse/issues/65926): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65822](https://github.com/ClickHouse/ClickHouse/issues/65822): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66322](https://github.com/ClickHouse/ClickHouse/issues/66322): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66449](https://github.com/ClickHouse/ClickHouse/issues/66449): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66717](https://github.com/ClickHouse/ClickHouse/issues/66717): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67320](https://github.com/ClickHouse/ClickHouse/issues/67320): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65080](https://github.com/ClickHouse/ClickHouse/issues/65080): Follow up to [#56541](https://github.com/ClickHouse/ClickHouse/issues/56541). [#57141](https://github.com/ClickHouse/ClickHouse/pull/57141) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#64997](https://github.com/ClickHouse/ClickHouse/issues/64997): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#65913](https://github.com/ClickHouse/ClickHouse/issues/65913): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66853](https://github.com/ClickHouse/ClickHouse/issues/66853): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#67072](https://github.com/ClickHouse/ClickHouse/issues/67072): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Update version after release. [#67691](https://github.com/ClickHouse/ClickHouse/pull/67691) ([robot-clickhouse](https://github.com/robot-clickhouse)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 71a4a722a36..183e1087c1c 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -55,6 +55,7 @@ v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 v23.8.16.40-lts 2024-08-02 +v23.8.16.16-lts 2024-08-20 v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 From 185038beb00617da3b825fc464efd694c179f9f8 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 16 Aug 2024 16:15:27 +0000 Subject: [PATCH 167/409] Add encrypted local named collections --- .../NamedCollectionsMetadataStorage.cpp | 111 +++++++++++++++++- .../NamedCollectionsMetadataStorage.h | 2 + 2 files changed, 108 insertions(+), 5 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 36191b89e86..ae9b7776c94 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,6 +16,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -74,9 +77,9 @@ public: }; -class NamedCollectionsMetadataStorage::LocalStorage : public INamedCollectionsStorage, private WithContext +class NamedCollectionsMetadataStorage::LocalStorage : public INamedCollectionsStorage, protected WithContext { -private: +protected: std::string root_path; public: @@ -168,7 +171,7 @@ public: return fs::remove(getPath(file_name)); } -private: +protected: std::string getPath(const std::string & file_name) const { const auto file_name_as_path = fs::path(file_name); @@ -178,6 +181,7 @@ private: return fs::path(root_path) / file_name_as_path; } +private: /// Delete .tmp files. They could be left undeleted in case of /// some exception or abrupt server restart. void cleanup() @@ -194,6 +198,97 @@ private: } }; +class NamedCollectionsMetadataStorage::LocalStorageEncrypted : public NamedCollectionsMetadataStorage::LocalStorage +{ +public: + LocalStorageEncrypted(ContextPtr context_, const std::string & path_) + : NamedCollectionsMetadataStorage::LocalStorage(context_, path_) + { + const auto & config = getContext()->getConfigRef(); + auto key_hex = config.getRawString("named_collections_storage.key_hex", ""); + try + { + key = boost::algorithm::unhex(key_hex); + key_fingerprint = FileEncryption::calculateKeyFingerprint(key); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length"); + } + + algorithm = FileEncryption::parseAlgorithmFromString(config.getString("named_collections_storage.algorithm", "aes_128_ctr")); + } + + std::string read(const std::string & file_name) const override + { + ReadBufferFromFile in(getPath(file_name)); + Memory<> encrypted_buffer(in.getFileSize()); + + FileEncryption::Header header; + try + { + header.read(in); + } + catch (Exception & e) + { + e.addMessage("While reading the header of encrypted file " + quoteString(file_name)); + throw; + } + + size_t bytes_read = 0; + while (bytes_read < encrypted_buffer.size() && !in.eof()) + { + bytes_read += in.read(encrypted_buffer.data() + bytes_read, encrypted_buffer.size() - bytes_read); + } + + std::string decrypted_buffer; + decrypted_buffer.resize(bytes_read); + FileEncryption::Encryptor encryptor(header.algorithm, key, header.init_vector); + encryptor.decrypt(encrypted_buffer.data(), bytes_read, decrypted_buffer.data()); + + LOG_DEBUG(getLogger("PMO"), "Read named collection {}: {}", file_name, decrypted_buffer); + return decrypted_buffer; + } + + void write(const std::string & file_name, const std::string & data, bool replace) override + { + if (!replace && fs::exists(file_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Metadata file {} for named collection already exists", + file_name); + } + + fs::create_directories(root_path); + + auto tmp_path = getPath(file_name + ".tmp"); + + auto out = std::make_unique(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); + FileEncryption::Header header{ + .algorithm = algorithm, + .key_fingerprint = key_fingerprint, + .init_vector = FileEncryption::InitVector::random() + }; + WriteBufferFromEncryptedFile out_encrypted(data.size(), std::move(out), key, header); + writeString(data, out_encrypted); + + out_encrypted.next(); + if (getContext()->getSettingsRef().fsync_metadata) + out_encrypted.sync(); + + LOG_DEBUG(getLogger("PMO"), "Wrote named collection {}: {} in plain text, encrypted {}", file_name, data, out_encrypted.buffer()); + + fs::rename(tmp_path, getPath(file_name)); + } + +private: + std::string key; + UInt128 key_fingerprint; + FileEncryption::Algorithm algorithm; +}; + + class NamedCollectionsMetadataStorage::ZooKeeperStorage : public INamedCollectionsStorage, private WithContext { @@ -495,7 +590,7 @@ std::unique_ptr NamedCollectionsMetadataStorage const auto & config = context_->getConfigRef(); const auto storage_type = config.getString(named_collections_storage_config_path + ".type", "local"); - if (storage_type == "local") + if (storage_type == "local" || storage_type == "local_encrypted") { const auto path = config.getString( named_collections_storage_config_path + ".path", @@ -504,7 +599,13 @@ std::unique_ptr NamedCollectionsMetadataStorage LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), "Using local storage for named collections at path: {}", path); - auto local_storage = std::make_unique(context_, path); + std::unique_ptr local_storage; + + if (storage_type == "local") + local_storage = std::make_unique(context_, path); + else if (storage_type == "local_encrypted") + local_storage = std::make_unique(context_, path); + return std::unique_ptr( new NamedCollectionsMetadataStorage(std::move(local_storage), context_)); } diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index c3468fbc468..db2b48fcd23 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -35,7 +35,9 @@ public: private: class INamedCollectionsStorage; class LocalStorage; + class LocalStorageEncrypted; class ZooKeeperStorage; + class ZooKeeperEncrypted; std::shared_ptr storage; From c269e0f71ce5198dfd7cce2d024164b452c5ea82 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 19 Aug 2024 10:13:49 +0000 Subject: [PATCH 168/409] Add encrypted named collections for ZooKeeper Consolidate the code so that Local and ZooKeeper storages use the same logic which uses memory buffers to encrypt and decrypt data. --- .../NamedCollectionsMetadataStorage.cpp | 225 ++++++++++-------- .../NamedCollectionsMetadataStorage.h | 2 +- 2 files changed, 123 insertions(+), 104 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index ae9b7776c94..b8269d2d55a 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -7,9 +7,10 @@ #include #include #include -#include -#include #include +#include +#include +#include #include #include #include @@ -129,6 +130,11 @@ public: ReadBufferFromFile in(getPath(file_name)); std::string data; readStringUntilEOF(data, in); + return readHook(data); + } + + virtual std::string readHook(const std::string & data) const + { return data; } @@ -145,8 +151,9 @@ public: fs::create_directories(root_path); auto tmp_path = getPath(file_name + ".tmp"); - WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(data, out); + auto write_data = writeHook(data); + WriteBufferFromFile out(tmp_path, write_data.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(write_data, out); out.next(); if (getContext()->getSettingsRef().fsync_metadata) @@ -156,6 +163,11 @@ public: fs::rename(tmp_path, getPath(file_name)); } + virtual std::string writeHook(const std::string & data) const + { + return data; + } + void remove(const std::string & file_name) override { if (!removeIfExists(file_name)) @@ -198,99 +210,7 @@ private: } }; -class NamedCollectionsMetadataStorage::LocalStorageEncrypted : public NamedCollectionsMetadataStorage::LocalStorage -{ -public: - LocalStorageEncrypted(ContextPtr context_, const std::string & path_) - : NamedCollectionsMetadataStorage::LocalStorage(context_, path_) - { - const auto & config = getContext()->getConfigRef(); - auto key_hex = config.getRawString("named_collections_storage.key_hex", ""); - try - { - key = boost::algorithm::unhex(key_hex); - key_fingerprint = FileEncryption::calculateKeyFingerprint(key); - } - catch (const std::exception &) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length"); - } - - algorithm = FileEncryption::parseAlgorithmFromString(config.getString("named_collections_storage.algorithm", "aes_128_ctr")); - } - - std::string read(const std::string & file_name) const override - { - ReadBufferFromFile in(getPath(file_name)); - Memory<> encrypted_buffer(in.getFileSize()); - - FileEncryption::Header header; - try - { - header.read(in); - } - catch (Exception & e) - { - e.addMessage("While reading the header of encrypted file " + quoteString(file_name)); - throw; - } - - size_t bytes_read = 0; - while (bytes_read < encrypted_buffer.size() && !in.eof()) - { - bytes_read += in.read(encrypted_buffer.data() + bytes_read, encrypted_buffer.size() - bytes_read); - } - - std::string decrypted_buffer; - decrypted_buffer.resize(bytes_read); - FileEncryption::Encryptor encryptor(header.algorithm, key, header.init_vector); - encryptor.decrypt(encrypted_buffer.data(), bytes_read, decrypted_buffer.data()); - - LOG_DEBUG(getLogger("PMO"), "Read named collection {}: {}", file_name, decrypted_buffer); - return decrypted_buffer; - } - - void write(const std::string & file_name, const std::string & data, bool replace) override - { - if (!replace && fs::exists(file_name)) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "Metadata file {} for named collection already exists", - file_name); - } - - fs::create_directories(root_path); - - auto tmp_path = getPath(file_name + ".tmp"); - - auto out = std::make_unique(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); - FileEncryption::Header header{ - .algorithm = algorithm, - .key_fingerprint = key_fingerprint, - .init_vector = FileEncryption::InitVector::random() - }; - WriteBufferFromEncryptedFile out_encrypted(data.size(), std::move(out), key, header); - writeString(data, out_encrypted); - - out_encrypted.next(); - if (getContext()->getSettingsRef().fsync_metadata) - out_encrypted.sync(); - - LOG_DEBUG(getLogger("PMO"), "Wrote named collection {}: {} in plain text, encrypted {}", file_name, data, out_encrypted.buffer()); - - fs::rename(tmp_path, getPath(file_name)); - } - -private: - std::string key; - UInt128 key_fingerprint; - FileEncryption::Algorithm algorithm; -}; - - - -class NamedCollectionsMetadataStorage::ZooKeeperStorage : public INamedCollectionsStorage, private WithContext +class NamedCollectionsMetadataStorage::ZooKeeperStorage : public INamedCollectionsStorage, protected WithContext { private: std::string root_path; @@ -370,18 +290,25 @@ public: std::string read(const std::string & file_name) const override { - return getClient()->get(getPath(file_name)); + auto data = getClient()->get(getPath(file_name)); + return readHook(data); + } + + virtual std::string readHook(const std::string & data) const + { + return data; } void write(const std::string & file_name, const std::string & data, bool replace) override { + auto write_data = writeHook(data); if (replace) { - getClient()->createOrUpdate(getPath(file_name), data, zkutil::CreateMode::Persistent); + getClient()->createOrUpdate(getPath(file_name), write_data, zkutil::CreateMode::Persistent); } else { - auto code = getClient()->tryCreate(getPath(file_name), data, zkutil::CreateMode::Persistent); + auto code = getClient()->tryCreate(getPath(file_name), write_data, zkutil::CreateMode::Persistent); if (code == Coordination::Error::ZNODEEXISTS) { @@ -393,6 +320,11 @@ public: } } + virtual std::string writeHook(const std::string & data) const + { + return data; + } + void remove(const std::string & file_name) override { getClient()->remove(getPath(file_name)); @@ -429,6 +361,89 @@ private: } }; +template +class NamedCollectionsMetadataStorageEncrypted : public BaseMetadataStorage +{ +public: + NamedCollectionsMetadataStorageEncrypted(ContextPtr context_, const std::string & path_) + : BaseMetadataStorage(context_, path_) + { + const auto & config = BaseMetadataStorage::getContext()->getConfigRef(); + auto key_hex = config.getRawString("named_collections_storage.key_hex", ""); + try + { + key = boost::algorithm::unhex(key_hex); + key_fingerprint = FileEncryption::calculateKeyFingerprint(key); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length"); + } + + algorithm = FileEncryption::parseAlgorithmFromString(config.getString("named_collections_storage.algorithm", "aes_128_ctr")); + } + + std::string readHook(const std::string & data) const override + { + ReadBufferFromString in(data); + Memory<> encrypted_buffer(data.length()); + + FileEncryption::Header header; + try + { + header.read(in); + } + catch (Exception & e) + { + e.addMessage("While reading the header of encrypted data"); + throw; + } + + size_t bytes_read = 0; + while (bytes_read < encrypted_buffer.size() && !in.eof()) + { + bytes_read += in.read(encrypted_buffer.data() + bytes_read, encrypted_buffer.size() - bytes_read); + } + + std::string decrypted_buffer; + decrypted_buffer.resize(bytes_read); + FileEncryption::Encryptor encryptor(header.algorithm, key, header.init_vector); + encryptor.decrypt(encrypted_buffer.data(), bytes_read, decrypted_buffer.data()); + + return decrypted_buffer; + } + + std::string writeHook(const std::string & data) const override + { + FileEncryption::Header header{ + .algorithm = algorithm, + .key_fingerprint = key_fingerprint, + .init_vector = FileEncryption::InitVector::random() + }; + + FileEncryption::Encryptor encryptor(header.algorithm, key, header.init_vector); + WriteBufferFromOwnString out; + header.write(out); + encryptor.encrypt(data.data(), data.size(), out); + return std::string(out.str()); + } + +private: + std::string key; + UInt128 key_fingerprint; + FileEncryption::Algorithm algorithm; +}; + +class NamedCollectionsMetadataStorage::LocalStorageEncrypted : public NamedCollectionsMetadataStorageEncrypted +{ + using NamedCollectionsMetadataStorageEncrypted::NamedCollectionsMetadataStorageEncrypted; +}; + +class NamedCollectionsMetadataStorage::ZooKeeperStorageEncrypted : public NamedCollectionsMetadataStorageEncrypted +{ + using NamedCollectionsMetadataStorageEncrypted::NamedCollectionsMetadataStorageEncrypted; +}; + NamedCollectionsMetadataStorage::NamedCollectionsMetadataStorage( std::shared_ptr storage_, ContextPtr context_) @@ -600,7 +615,6 @@ std::unique_ptr NamedCollectionsMetadataStorage "Using local storage for named collections at path: {}", path); std::unique_ptr local_storage; - if (storage_type == "local") local_storage = std::make_unique(context_, path); else if (storage_type == "local_encrypted") @@ -609,10 +623,15 @@ std::unique_ptr NamedCollectionsMetadataStorage return std::unique_ptr( new NamedCollectionsMetadataStorage(std::move(local_storage), context_)); } - if (storage_type == "zookeeper" || storage_type == "keeper") + if (storage_type == "zookeeper" || storage_type == "keeper" || storage_type == "zookeeper_encrypted" || storage_type == "keeper_encrypted") { const auto path = config.getString(named_collections_storage_config_path + ".path"); - auto zk_storage = std::make_unique(context_, path); + + std::unique_ptr zk_storage; + if (storage_type == "zookeeper" || storage_type == "keeper") + zk_storage = std::make_unique(context_, path); + else if (storage_type == "zookeeper_encrypted" || storage_type == "keeper_encrypted") + zk_storage = std::make_unique(context_, path); LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), "Using zookeeper storage for named collections at path: {}", path); diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index db2b48fcd23..52805e8359d 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -37,7 +37,7 @@ private: class LocalStorage; class LocalStorageEncrypted; class ZooKeeperStorage; - class ZooKeeperEncrypted; + class ZooKeeperStorageEncrypted; std::shared_ptr storage; From 4b05106cc40b09f354d3717ece3850935157bbe6 Mon Sep 17 00:00:00 2001 From: PHO Date: Thu, 1 Aug 2024 14:07:49 +0900 Subject: [PATCH 169/409] Fix the upper bound of function fromModifiedJulianDay() The upper bound was supposed to be 9999-12-31 but it was accidentally set to 9999-01-01. --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/zh/sql-reference/functions/date-time-functions.md | 2 +- src/Functions/GregorianDate.cpp | 4 ++-- .../queries/0_stateless/01544_fromModifiedJulianDay.reference | 2 ++ tests/queries/0_stateless/01544_fromModifiedJulianDay.sql | 2 ++ 5 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4f5e5a5d716..3d95ae2cb74 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -4287,7 +4287,7 @@ Result: ## fromModifiedJulianDay -Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number to a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD`. This function supports day number from `-678941` to `2973119` (which represent 0000-01-01 and 9999-12-31 respectively). It raises an exception if the day number is outside of the supported range. +Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number to a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD`. This function supports day number from `-678941` to `2973483` (which represent 0000-01-01 and 9999-12-31 respectively). It raises an exception if the day number is outside of the supported range. **Syntax** diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index 18b9f3495c0..9fa2d79c655 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -1157,7 +1157,7 @@ SELECT toModifiedJulianDayOrNull('2020-01-01'); ## fromModifiedJulianDay {#frommodifiedjulianday} -将 [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) 数字转换为 `YYYY-MM-DD` 文本格式的 [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) 日期。该函数支持从 `-678941` 到 `2973119` 的天数(分别代表 0000-01-01 和 9999-12-31)。如果天数超出支持范围,则会引发异常。 +将 [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) 数字转换为 `YYYY-MM-DD` 文本格式的 [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) 日期。该函数支持从 `-678941` 到 `2973483` 的天数(分别代表 0000-01-01 和 9999-12-31)。如果天数超出支持范围,则会引发异常。 **语法** diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index 91861e8bbd2..82c81d2bb4f 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -284,12 +284,12 @@ void OrdinalDate::init(int64_t modified_julian_day) bool OrdinalDate::tryInit(int64_t modified_julian_day) { - /// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively). + /// This function supports day number from -678941 to 2973483 (which represent 0000-01-01 and 9999-12-31 respectively). if (modified_julian_day < -678941) return false; - if (modified_julian_day > 2973119) + if (modified_julian_day > 2973483) return false; const auto a = modified_julian_day + 678575; diff --git a/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference b/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference index 443b90b80a5..4c6d75346e1 100644 --- a/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference +++ b/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference @@ -4,6 +4,8 @@ Invocation with constant 2020-11-01 \N \N +0000-01-01 +9999-12-31 or null 2020-11-01 \N diff --git a/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql b/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql index d405aa16f3f..6f0f08c363f 100644 --- a/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql +++ b/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql @@ -7,6 +7,8 @@ SELECT fromModifiedJulianDay(59154); SELECT fromModifiedJulianDay(NULL); SELECT fromModifiedJulianDay(CAST(NULL, 'Nullable(Int64)')); SELECT fromModifiedJulianDay(-678942); -- { serverError CANNOT_FORMAT_DATETIME } +SELECT fromModifiedJulianDay(-678941); +SELECT fromModifiedJulianDay(2973483); SELECT fromModifiedJulianDay(2973484); -- { serverError CANNOT_FORMAT_DATETIME } SELECT 'or null'; From 0ccbb554b9d0b7055415569559e029060261243e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 20 Aug 2024 10:58:14 +0200 Subject: [PATCH 170/409] Update 02995_index_7.sh --- tests/queries/0_stateless/02995_index_7.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02995_index_7.sh b/tests/queries/0_stateless/02995_index_7.sh index a5fdd98b2f8..7a03b0d4c1a 100755 --- a/tests/queries/0_stateless/02995_index_7.sh +++ b/tests/queries/0_stateless/02995_index_7.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From abf556d641f6a29b32dbe4f24fec7a8ae90990b8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 20 Aug 2024 11:39:47 +0200 Subject: [PATCH 171/409] review changes --- .../functions/type-conversion-functions.md | 132 +++++++++--------- 1 file changed, 66 insertions(+), 66 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index a03394be226..8acb2cd30b1 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4878,7 +4878,7 @@ toIntervalYear(n) **Arguments** -- `n` — Number of years. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of years. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -4890,17 +4890,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalYear(1) AS interval_to_year -SELECT date + interval_to_year +SELECT date + interval_to_year AS result ``` Result: ```response -┌─plus(date, interval_to_year)─┐ -│ 2020-01-01 │ -└──────────────────────────────┘ +┌─────result─┐ +│ 2025-06-15 │ +└────────────┘ ``` ## toIntervalQuarter @@ -4915,7 +4915,7 @@ toIntervalQuarter(n) **Arguments** -- `n` — Number of quarters. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of quarters. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -4927,17 +4927,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalQuarter(1) AS interval_to_quarter -SELECT date + interval_to_quarter +SELECT date + interval_to_quarter AS result ``` Result: ```response -┌─plus(date, interval_to_quarter)─┐ -│ 2019-04-01 │ -└─────────────────────────────────┘ +┌─────result─┐ +│ 2024-09-15 │ +└────────────┘ ``` ## toIntervalMonth @@ -4952,7 +4952,7 @@ toIntervalMonth(n) **Arguments** -- `n` — Number of months. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of months. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -4964,17 +4964,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalMonth(1) AS interval_to_month -SELECT date + interval_to_month +SELECT date + interval_to_month AS result ``` Result: ```response -┌─plus(date, interval_to_month)─┐ -│ 2019-02-01 │ -└───────────────────────────────┘ +┌─────result─┐ +│ 2024-07-15 │ +└────────────┘ ``` ## toIntervalWeek @@ -4989,7 +4989,7 @@ toIntervalWeek(n) **Arguments** -- `n` — Number of weeks. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of weeks. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5001,17 +5001,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalWeek(1) AS interval_to_week -SELECT date + interval_to_week +SELECT date + interval_to_week AS result ``` Result: ```response -┌─plus(date, interval_to_week)─┐ -│ 2019-01-08 │ -└──────────────────────────────┘ +┌─────result─┐ +│ 2024-06-22 │ +└────────────┘ ``` ## toIntervalDay @@ -5026,7 +5026,7 @@ toIntervalDay(n) **Arguments** -- `n` — Number of days. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of days. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5038,17 +5038,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalDay(5) AS interval_to_days -SELECT date + interval_to_days +SELECT date + interval_to_days AS result ``` Result: ```response -┌─plus(date, interval_to_days)─┐ -│ 2019-01-06 │ -└──────────────────────────────┘ +┌─────result─┐ +│ 2024-06-20 │ +└────────────┘ ``` ## toIntervalHour @@ -5063,7 +5063,7 @@ toIntervalHour(n) **Arguments** -- `n` — Number of hours. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of hours. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5075,17 +5075,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalHour(12) AS interval_to_hours -SELECT date + interval_to_hours +SELECT date + interval_to_hours AS result ``` Result: ```response -┌─plus(date, interval_to_hours)─┐ -│ 2019-01-01 12:00:00 │ -└───────────────────────────────┘ +┌──────────────result─┐ +│ 2024-06-15 12:00:00 │ +└─────────────────────┘ ``` ## toIntervalMinute @@ -5100,7 +5100,7 @@ toIntervalMinute(n) **Arguments** -- `n` — Number of minutes. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of minutes. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5112,17 +5112,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalMinute(12) AS interval_to_minutes -SELECT date + interval_to_minutes +SELECT date + interval_to_minutes AS result ``` Result: ```response -┌─plus(date, interval_to_minutes)─┐ -│ 2019-01-01 00:12:00 │ -└─────────────────────────────────┘ +┌──────────────result─┐ +│ 2024-06-15 00:12:00 │ +└─────────────────────┘ ``` ## toIntervalSecond @@ -5137,7 +5137,7 @@ toIntervalSecond(n) **Arguments** -- `n` — Number of seconds. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of seconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5149,17 +5149,17 @@ Query: ``` sql WITH - toDate('2019-01-01') AS date, + toDate('2024-06-15') AS date, toIntervalSecond(30) AS interval_to_seconds -SELECT date + interval_to_seconds +SELECT date + interval_to_seconds AS result ``` Result: ```response -┌─plus(date, interval_to_seconds)─┐ -│ 2019-01-01 00:00:30 │ -└─────────────────────────────────┘ +┌──────────────result─┐ +│ 2024-06-15 00:00:30 │ +└─────────────────────┘ ``` ## toIntervalMillisecond @@ -5174,7 +5174,7 @@ toIntervalMillisecond(n) **Arguments** -- `n` — Number of milliseconds. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of milliseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5186,17 +5186,17 @@ Query: ``` sql WITH - toDateTime('2019-01-01') AS date, + toDateTime('2024-06-15') AS date, toIntervalMillisecond(30) AS interval_to_milliseconds -SELECT date + interval_to_milliseconds +SELECT date + interval_to_milliseconds AS result ``` Result: ```response -┌─plus(date, interval_to_milliseconds)─┐ -│ 2019-01-01 00:00:00.030 │ -└──────────────────────────────────────┘ +┌──────────────────result─┐ +│ 2024-06-15 00:00:00.030 │ +└─────────────────────────┘ ``` ## toIntervalMicrosecond @@ -5211,7 +5211,7 @@ toIntervalMicrosecond(n) **Arguments** -- `n` — Number of microseconds. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of microseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5223,17 +5223,17 @@ Query: ``` sql WITH - toDateTime('2019-01-01') AS date, + toDateTime('2024-06-15') AS date, toIntervalMicrosecond(30) AS interval_to_microseconds -SELECT date + interval_to_microseconds +SELECT date + interval_to_microseconds AS result ``` Result: ```response -┌─plus(date, interval_to_microseconds)─┐ -│ 2019-01-01 00:00:00.000030 │ -└──────────────────────────────────────┘ +┌─────────────────────result─┐ +│ 2024-06-15 00:00:00.000030 │ +└────────────────────────────┘ ``` ## toIntervalNanosecond @@ -5248,7 +5248,7 @@ toIntervalNanosecond(n) **Arguments** -- `n` — Number of nanoseconds. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of nanoseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). **Returned values** @@ -5260,17 +5260,17 @@ Query: ``` sql WITH - toDateTime('2019-01-01') AS date, + toDateTime('2024-06-15') AS date, toIntervalNanosecond(30) AS interval_to_nanoseconds -SELECT date + interval_to_nanoseconds +SELECT date + interval_to_nanoseconds AS result ``` Result: ```response -┌─plus(date, interval_to_nanoseconds)─┐ -│ 2019-01-01 00:00:00.000000030 │ -└─────────────────────────────────────┘ +┌────────────────────────result─┐ +│ 2024-06-15 00:00:00.000000030 │ +└───────────────────────────────┘ ``` ## parseDateTime From 68f95f2bd1197f3094b019a9ebf0b77d6a619afc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 20 Aug 2024 10:29:57 +0000 Subject: [PATCH 172/409] Update version_date.tsv and changelogs after v24.3.8.13-lts --- docs/changelogs/v24.3.8.13-lts.md | 16 ++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 17 insertions(+) create mode 100644 docs/changelogs/v24.3.8.13-lts.md diff --git a/docs/changelogs/v24.3.8.13-lts.md b/docs/changelogs/v24.3.8.13-lts.md new file mode 100644 index 00000000000..6fbceacd624 --- /dev/null +++ b/docs/changelogs/v24.3.8.13-lts.md @@ -0,0 +1,16 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.8.13-lts (84bbfc70f5d) FIXME as compared to v24.3.7.30-lts (c8a28cf4331) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68562](https://github.com/ClickHouse/ClickHouse/issues/68562): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68114](https://github.com/ClickHouse/ClickHouse/issues/68114): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67989](https://github.com/ClickHouse/ClickHouse/issues/67989): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68335](https://github.com/ClickHouse/ClickHouse/issues/68335): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68392](https://github.com/ClickHouse/ClickHouse/issues/68392): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 183e1087c1c..776a53ec01c 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -13,6 +13,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 v24.3.5.46-lts 2024-07-03 From 0407171fce808a6eafee8d46e517008d74109d64 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 20 Aug 2024 12:53:31 +0200 Subject: [PATCH 173/409] update argument types and fix style --- .../functions/type-conversion-functions.md | 22 +++++++++--------- .../aspell-ignore/en/aspell-dict.txt | 23 +++++++++++++++++++ 2 files changed, 34 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 8acb2cd30b1..d6b93d8ecdb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4878,7 +4878,7 @@ toIntervalYear(n) **Arguments** -- `n` — Number of years. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of years. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -4915,7 +4915,7 @@ toIntervalQuarter(n) **Arguments** -- `n` — Number of quarters. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of quarters. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -4952,7 +4952,7 @@ toIntervalMonth(n) **Arguments** -- `n` — Number of months. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of months. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -4989,7 +4989,7 @@ toIntervalWeek(n) **Arguments** -- `n` — Number of weeks. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of weeks. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5026,7 +5026,7 @@ toIntervalDay(n) **Arguments** -- `n` — Number of days. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of days. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5063,7 +5063,7 @@ toIntervalHour(n) **Arguments** -- `n` — Number of hours. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of hours. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5100,7 +5100,7 @@ toIntervalMinute(n) **Arguments** -- `n` — Number of minutes. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of minutes. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5137,7 +5137,7 @@ toIntervalSecond(n) **Arguments** -- `n` — Number of seconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of seconds. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5174,7 +5174,7 @@ toIntervalMillisecond(n) **Arguments** -- `n` — Number of milliseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of milliseconds. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5211,7 +5211,7 @@ toIntervalMicrosecond(n) **Arguments** -- `n` — Number of microseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of microseconds. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** @@ -5248,7 +5248,7 @@ toIntervalNanosecond(n) **Arguments** -- `n` — Number of nanoseconds. Positive integer number or string representation thereof. [Int*](../data-types/int-uint.md)/[String](../data-types/string.md). +- `n` — Number of nanoseconds. Integer numbers or string representations thereof, and float numbers. [(U)Int*](../data-types/int-uint.md)/[Float*](../data-types/float.md)/[String](../data-types/string.md). **Returned values** diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b21ae0764c6..f4fe336f0fd 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -384,6 +384,18 @@ IntelliJ IntelliSense InterserverConnection InterserverThreads +IntervalDay +IntervalHour +IntervalMicrosecond +IntervalMillisecond +IntervalMilliseconds +IntervalMinute +IntervalMonth +IntervalNanosecond +IntervalQuarter +IntervalSecond +IntervalWeek +IntervalYear IsPentagon IsResClassIII IsValid @@ -2715,6 +2727,17 @@ toISOWeek toISOYear toInt toInterval +toIntervalDay +toIntervalHour +toIntervalMicrosecond +toIntervalMillisecond +toIntervalMinute +toIntervalMonth +toIntervalNanosecond +toIntervalQuarter +toIntervalSecond +toIntervalWeek +toIntervalYear toJSONString toLastDayOfMonth toLastDayOfWeek From 08cb7ff8d866364013286043a8810329089b5832 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 20 Aug 2024 19:02:37 +0800 Subject: [PATCH 174/409] Add integration tests and delete stateless tests --- .../__init__.py | 0 .../configs/config.d/cluster.xml | 12 ++++ .../configs/config.xml | 9 +++ .../test_incorrect_datetime_format/test.py | 57 +++++++++++++++++++ ...fix_datetime_implicit_conversion.reference | 1 - ...03215_fix_datetime_implicit_conversion.sql | 15 ----- 6 files changed, 78 insertions(+), 16 deletions(-) create mode 100644 tests/integration/test_incorrect_datetime_format/__init__.py create mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml create mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.xml create mode 100644 tests/integration/test_incorrect_datetime_format/test.py delete mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference delete mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql diff --git a/tests/integration/test_incorrect_datetime_format/__init__.py b/tests/integration/test_incorrect_datetime_format/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml new file mode 100644 index 00000000000..9c7f02c190f --- /dev/null +++ b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml @@ -0,0 +1,12 @@ + + + + + + node1 + 9000 + + + + + diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.xml b/tests/integration/test_incorrect_datetime_format/configs/config.xml new file mode 100644 index 00000000000..053b5d30418 --- /dev/null +++ b/tests/integration/test_incorrect_datetime_format/configs/config.xml @@ -0,0 +1,9 @@ + + + information + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py new file mode 100644 index 00000000000..eb6501fbec1 --- /dev/null +++ b/tests/integration/test_incorrect_datetime_format/test.py @@ -0,0 +1,57 @@ +import logging +import pytest +from helpers.cluster import ClickHouseCluster + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=[ + "configs/config.d/cluster.xml", + ] + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_incorrect_datetime_format(cluster): + """ + Test for an MSan issue which is caused by parsing incorrect datetime string + """ + + node = cluster.instances["node"] + + table_name = "test_delete_race_leftovers" + additional_settings = { + # use another disk not to interfere with other tests + "storage_policy": "one_disk", + # always remove parts in parallel + "concurrent_part_removal_threshold": 1, + } + + node.query(""" + CREATE TABLE tab + ( + a DateTime, + pk String + ) Engine = MergeTree() ORDER BY pk; + """ + ) + + res = node.query("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'").strip() + assert res == "0" + + error = node.query_and_get_error("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'").strip() + print(error) + assert "Cannot parse time component of DateTime 09:58:0" in error + + error = node.query_and_get_error("SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'").strip() + print(error) + assert "Cannot convert string '2024-08-0 09:58:09' to type DateTime" in error diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference deleted file mode 100644 index 573541ac970..00000000000 --- a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql deleted file mode 100644 index 70a8a3432a6..00000000000 --- a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql +++ /dev/null @@ -1,15 +0,0 @@ -DROP TABLE IF EXISTS tab SYNC; - -CREATE TABLE tab -( - a DateTime, - pk String -) Engine = MergeTree() ORDER BY pk; - -INSERT INTO tab select cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 1; - -SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'; -SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } -SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'; -- { serverError TYPE_MISMATCH } - -DROP TABLE IF EXISTS tab SYNC; From 0563e19cf336c629312bbc23f39c6dd8676238b5 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 20 Aug 2024 19:09:27 +0800 Subject: [PATCH 175/409] Little fix --- .../configs/config.d/cluster.xml | 3 +-- tests/integration/test_incorrect_datetime_format/test.py | 8 -------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml index 9c7f02c190f..a27968fb3d2 100644 --- a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml +++ b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml @@ -3,8 +3,7 @@ - node1 - 9000 + node diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py index eb6501fbec1..c1803aa95a1 100644 --- a/tests/integration/test_incorrect_datetime_format/test.py +++ b/tests/integration/test_incorrect_datetime_format/test.py @@ -28,14 +28,6 @@ def test_incorrect_datetime_format(cluster): node = cluster.instances["node"] - table_name = "test_delete_race_leftovers" - additional_settings = { - # use another disk not to interfere with other tests - "storage_policy": "one_disk", - # always remove parts in parallel - "concurrent_part_removal_threshold": 1, - } - node.query(""" CREATE TABLE tab ( From 6a405b4aab1ce3f23555e2217856faecb414afca Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Aug 2024 13:14:21 +0200 Subject: [PATCH 176/409] Update 01825_new_type_json_ghdata_insert_select.sh --- .../0_stateless/01825_new_type_json_ghdata_insert_select.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh index 3f5fc91f8fc..b450e9827c2 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-s3-storage, long -# ^ no-s3-storage: it is memory-hungry +# Tags: no-fasttest, no-s3-storage, long, no-asan +# ^ no-s3-storage: it is memory-hungry, no-asan: too long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 653c978dfa30bf1807912eccf8ba9a8d1f7a4da5 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 19 Aug 2024 12:08:23 +0200 Subject: [PATCH 177/409] autorelease to check builds and functional tests --- .github/actions/clean/action.yml | 20 ++++++++++--- .github/workflows/auto_releases.yml | 34 ++++++++++++++-------- tests/ci/artifactory.py | 2 ++ tests/ci/auto_release.py | 44 +++++++++++++++++++++-------- tests/ci/ci_utils.py | 2 +- 5 files changed, 74 insertions(+), 28 deletions(-) diff --git a/.github/actions/clean/action.yml b/.github/actions/clean/action.yml index 547738b17cc..8c22523cacf 100644 --- a/.github/actions/clean/action.yml +++ b/.github/actions/clean/action.yml @@ -1,11 +1,23 @@ name: Clean runner description: Clean the runner's temp path on ending +inputs: + images: + description: clean docker images + default: false + type: boolean runs: using: "composite" steps: - - name: Clean + - name: Clean Temp shell: bash run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "${{runner.temp}}" + sudo rm -fr "${{runner.temp}}" + - name: Clean Docker Containers + shell: bash + run: | + docker rm -vf $(docker ps -aq) ||: + - name: Clean Docker Images + if: ${{ inputs.images }} + shell: bash + run: | + docker rmi -f $(docker images -aq) ||: diff --git a/.github/workflows/auto_releases.yml b/.github/workflows/auto_releases.yml index 28483ea136f..2fdf4e30a70 100644 --- a/.github/workflows/auto_releases.yml +++ b/.github/workflows/auto_releases.yml @@ -14,7 +14,7 @@ on: dry-run: description: 'Dry run' required: false - default: true + default: false type: boolean jobs: @@ -51,7 +51,11 @@ jobs: cat /tmp/autorelease_params.json echo 'EOF' } >> "$GITHUB_OUTPUT" - echo "DRY_RUN=true" >> "$GITHUB_OUTPUT" + if [[ "${{ github.event_name }}" == "schedule" ]]; then + echo "DRY_RUN=true" >> "$GITHUB_OUTPUT" + else + echo "DRY_RUN=${{ github.event.inputs.dry-run }}" >> "$GITHUB_OUTPUT" + fi - name: Post Release Branch statuses run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -74,14 +78,22 @@ jobs: secrets: ROBOT_CLICKHOUSE_COMMIT_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - PostSlackMessage: - needs: [AutoReleaseInfo] + CleanUp: + needs: [Releases] runs-on: [self-hosted, release-maker] - if: ${{ !cancelled() }} steps: - - name: Check out repository code - uses: ClickHouse/checkout@v1 - - name: Post - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} + - uses: ./.github/actions/clean + with: + images: true + +# PostSlackMessage: +# needs: [Releases] +# runs-on: [self-hosted, release-maker] +# if: ${{ !cancelled() }} +# steps: +# - name: Check out repository code +# uses: ClickHouse/checkout@v1 +# - name: Post +# run: | +# cd "$GITHUB_WORKSPACE/tests/ci" +# python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index f3d7d24f717..9457fa32ad3 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -143,6 +143,8 @@ class DebianArtifactory: print(f" {cmd}") Shell.check(cmd, strict=True) Shell.check("sync") + time.sleep(10) + Shell.check(f"lsof +D R2MountPoint.MOUNT_POINT", verbose=True) def test_packages(self): Shell.check("docker pull ubuntu:latest", strict=True) diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 58cfc833afe..89714b2fb4b 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,5 +1,4 @@ import argparse -import copy import dataclasses import json import os @@ -77,8 +76,10 @@ class AutoReleaseInfo: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) # dump file for GH action matrix that is similar to the file above but with dropped not ready release branches - params = copy.deepcopy(self) - params.releases = [release for release in params.releases if release.ready] + params = dataclasses.asdict(self) + params["releases"] = [ + release for release in params["releases"] if release["ready"] + ] with open(AUTORELEASE_MATRIX_PARAMS, "w", encoding="utf-8") as f: print(json.dumps(params, indent=2), file=f) @@ -110,7 +111,6 @@ def _prepare(token): refs = list(repo.get_git_matching_refs(f"tags/v{pr.head.ref}")) assert refs - refs.sort(key=lambda ref: ref.ref) latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) @@ -118,6 +118,10 @@ def _prepare(token): f"git rev-list --first-parent {latest_release_tag.tag}..origin/{pr.head.ref}", ).split("\n") commit_num = len(commits) + if latest_release_tag.tag.endswith("new"): + print("It's a new release branch - skip auto release for it") + continue + print( f"Previous release [{latest_release_tag.tag}] was [{commit_num}] commits ago, date [{latest_release_tag.tagger.date}]" ) @@ -141,17 +145,33 @@ def _prepare(token): commits_to_branch_head += 1 continue - commit_ci_status = CI.GH.get_commit_status_by_name( - token=token, - commit_sha=commit, - # handle old name for old releases - status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), - ) + # TODO: switch to check if CI is entirely green + statuses = [ + CI.GH.get_commit_status_by_name( + token=token, + commit_sha=commit, + # handle old name for old releases + status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), + ), + CI.GH.get_commit_status_by_name( + token=token, + commit_sha=commit, + # handle old name for old releases + status_name=CI.JobNames.STATELESS_TEST_RELEASE, + ), + CI.GH.get_commit_status_by_name( + token=token, + commit_sha=commit, + # handle old name for old releases + status_name=CI.JobNames.STATEFUL_TEST_RELEASE, + ), + ] commit_sha = commit - if commit_ci_status == SUCCESS: + if any(status == SUCCESS for status in statuses): + commit_ci_status = SUCCESS break - print(f"CI status [{commit_ci_status}] - skip") + print(f"CI status [{statuses}] - skip") commits_to_branch_head += 1 ready = False diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 97ab10f1b58..a4c0977f47c 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -118,7 +118,7 @@ class GH: statuses = response.json() for status in statuses: if status["context"] in status_name: - return status["state"] + return status["state"] # type: ignore # Check if there is a next page url = response.links.get("next", {}).get("url") From 24eeaffa7a3ddbfa0fb7bc4546942bc18cab06af Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 20 Aug 2024 14:02:09 +0200 Subject: [PATCH 178/409] init --- src/Storages/VirtualColumnUtils.cpp | 11 +++++----- .../03203_hive_style_partitioning.reference | 1 + .../03203_hive_style_partitioning.sh | 19 +++++++++--------- .../partitioning/a=b/a=b/sample.parquet | Bin 0 -> 1308 bytes 4 files changed, 17 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/data_hive/partitioning/a=b/a=b/sample.parquet diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index d932f5cc469..edf50907752 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -136,14 +136,15 @@ std::unordered_map parseHivePartitioningKeysAndValues( std::unordered_map key_values; std::string key, value; - std::unordered_set used_keys; + std::unordered_map used_keys; while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) { - if (used_keys.contains(key)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {}, only unique keys are allowed", path, key); - used_keys.insert(key); + auto it = used_keys.find(key); + if (it != used_keys.end() && it->second != value) + throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {} with different values, only unique keys are allowed", path, key); + used_keys.insert({key, value}); - auto col_name = "_" + key; + auto col_name = key; while (storage_columns.has(col_name)) col_name = "_" + col_name; key_values[col_name] = value; diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index a4a2e48e046..12ffd17c102 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -39,6 +39,7 @@ Array(Int64) LowCardinality(Float64) 2070 1 1 +b TESTING THE URL PARTITIONING first last Elizabeth Jorge Frank Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index db1f073d736..5a0bd482985 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -27,26 +27,27 @@ SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/c SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; -SELECT _number, _date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; -SELECT _array, _float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; -SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; -SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; +SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; +SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT toTypeName(array), toTypeName(float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE number = 42; """ $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; -SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; +SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; +SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "INCORRECT_DATA" $CLICKHOUSE_LOCAL -n -q """ @@ -78,7 +79,7 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" +SELECT *, non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; @@ -109,7 +110,7 @@ SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; """ diff --git a/tests/queries/0_stateless/data_hive/partitioning/a=b/a=b/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/a=b/a=b/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 From 99e8a0babc04c9bae0565977090bdbd01272de0e Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Tue, 20 Aug 2024 14:52:54 +0200 Subject: [PATCH 179/409] materialized_view_deduplication performance comparison test performance comparison test to check deduplication in MATERIALIZED VIEW's. Logic is similar to, but with a bigger insert tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql --- .../materialized_view_deduplication.xml | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 tests/performance/materialized_view_deduplication.xml diff --git a/tests/performance/materialized_view_deduplication.xml b/tests/performance/materialized_view_deduplication.xml new file mode 100644 index 00000000000..621effd23dd --- /dev/null +++ b/tests/performance/materialized_view_deduplication.xml @@ -0,0 +1,34 @@ + + + 1 + + + + CREATE TABLE dst (`key` Int64, `value` String) + ENGINE = MergeTree ORDER BY tuple() + SETTINGS non_replicated_deduplication_window=1000; + + + CREATE TABLE mv_dst (`key` Int64, `value` String) + ENGINE = MergeTree ORDER BY tuple() + SETTINGS non_replicated_deduplication_window=1000; + + + CREATE MATERIALIZED VIEW mv_first TO mv_dst + AS SELECT 0 AS key, value AS value FROM dst; + + + CREATE MATERIALIZED VIEW mv_second TO mv_dst + AS SELECT 0 AS key, value AS value FROM dst; + + INSERT INTO dst SELECT number as key, toString(number) from numbers(1000); + + + INSERT INTO dst SELECT number as key, toString(number) from numbers(1000); + + + DROP TABLE IF EXISTS dst + DROP TABLE IF EXISTS mv_dst + DROP TABLE IF EXISTS mv_first + DROP TABLE IF EXISTS mv_second + From ee9080fba714fbe9ac482b4268ed164579ebda89 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 20 Aug 2024 15:11:06 +0200 Subject: [PATCH 180/409] Revert "Fix unexpected behavior with `FORMAT` and `SETTINGS` parsing" --- programs/client/Client.cpp | 3 - programs/server/Server.cpp | 2 +- src/Access/AccessControl.cpp | 8 +-- src/Access/AccessControl.h | 5 +- src/Access/SettingsConstraints.cpp | 8 +-- src/Client/ClientBase.cpp | 60 ++++++++++++----- src/Interpreters/InterpreterSetQuery.cpp | 34 +++++----- src/Interpreters/InterpreterSetQuery.h | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 67 +++++++------------ ...QueryWithOutputSettingsPushDownVisitor.cpp | 56 ++++++++++++++++ .../QueryWithOutputSettingsPushDownVisitor.h | 39 +++++++++++ .../00857_global_joinsavel_table_alias.sql | 1 + .../01401_FORMAT_SETTINGS.reference | 4 +- .../0_stateless/01401_FORMAT_SETTINGS.sh | 2 +- .../03003_compatibility_setting_bad_value.sql | 3 +- .../03172_format_settings_clauses.reference | 14 ---- .../03172_format_settings_clauses.sql | 30 --------- 17 files changed, 197 insertions(+), 141 deletions(-) create mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp create mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.h delete mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.reference delete mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 39edaf3497e..25c94c56aa6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1164,9 +1164,6 @@ void Client::processOptions(const OptionsDescription & options_description, /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) client_context = global_context; initClientContext(); - - /// Allow to pass-through unknown settings to the server. - client_context->getAccessControl().allowAllSettings(); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16fee378cf0..74228fae5a3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1922,7 +1922,7 @@ try auto & access_control = global_context->getAccessControl(); try { - access_control.setupFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); + access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); } catch (...) { diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index d4f8c7bc859..95a467bbbe5 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -280,7 +280,7 @@ void AccessControl::shutdown() } -void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, +void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_) { if (config_.has("custom_settings_prefixes")) @@ -868,10 +868,4 @@ const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const return *external_authenticators; } - -void AccessControl::allowAllSettings() -{ - custom_settings_prefixes->registerPrefixes({""}); -} - } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 7d8ee1232d0..bfaf256ad48 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -57,7 +57,7 @@ public: void shutdown() override; /// Initializes access storage (user directories). - void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, + void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_); /// Parses access entities from a configuration loaded from users.xml. @@ -238,9 +238,6 @@ public: /// Gets manager of notifications. AccessChangesNotifier & getChangesNotifier(); - /// Allow all setting names - this can be used in clients to pass-through unknown settings to the server. - void allowAllSettings(); - private: class ContextAccessCache; class CustomSettingsPrefixes; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 7506e365035..a274f6b54f2 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -219,8 +219,8 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang }); } -template -bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) +template +bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) { Field current_value; bool has_current_value = current_settings.tryGet(change.name, current_value); @@ -230,12 +230,12 @@ bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & chan return false; if (throw_on_failure) - new_value = SettingsT::castValueUtil(change.name, change.value); + new_value = T::castValueUtil(change.name, change.value); else { try { - new_value = SettingsT::castValueUtil(change.name, change.value); + new_value = T::castValueUtil(change.name, change.value); } catch (...) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e312f2a8158..01d03006eec 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -58,7 +58,6 @@ #include #include #include -#include #include #include #include @@ -1609,14 +1608,14 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des auto metadata = storage->getInMemoryMetadataPtr(); QueryPlan plan; storage->read( - plan, - sample.getNames(), - storage->getStorageSnapshot(metadata, client_context), - query_info, - client_context, - {}, - client_context->getSettingsRef().max_block_size, - getNumberOfPhysicalCPUCores()); + plan, + sample.getNames(), + storage->getStorageSnapshot(metadata, client_context), + query_info, + client_context, + {}, + client_context->getSettingsRef().max_block_size, + getNumberOfPhysicalCPUCores()); auto builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(client_context), @@ -1893,19 +1892,48 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin profile_events.watch.restart(); { - /// Temporarily apply query settings to the context. - Settings old_settings = client_context->getSettingsCopy(); - SCOPE_EXIT_SAFE( - { - client_context->setSettings(old_settings); + /// Temporarily apply query settings to context. + std::optional old_settings; + SCOPE_EXIT_SAFE({ + if (old_settings) + client_context->setSettings(*old_settings); }); - InterpreterSetQuery::applySettingsFromQuery(parsed_query, client_context); + + auto apply_query_settings = [&](const IAST & settings_ast) + { + if (!old_settings) + old_settings.emplace(client_context->getSettingsRef()); + client_context->applySettingsChanges(settings_ast.as()->changes); + client_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); + }; + + const auto * insert = parsed_query->as(); + if (const auto * select = parsed_query->as(); select && select->settings()) + apply_query_settings(*select->settings()); + else if (const auto * select_with_union = parsed_query->as()) + { + const ASTs & children = select_with_union->list_of_selects->children; + if (!children.empty()) + { + // On the client it is enough to apply settings only for the + // last SELECT, since the only thing that is important to apply + // on the client is format settings. + const auto * last_select = children.back()->as(); + if (last_select && last_select->settings()) + { + apply_query_settings(*last_select->settings()); + } + } + } + else if (const auto * query_with_output = parsed_query->as(); query_with_output && query_with_output->settings_ast) + apply_query_settings(*query_with_output->settings_ast); + else if (insert && insert->settings_ast) + apply_query_settings(*insert->settings_ast); if (!connection->checkConnected(connection_parameters.timeouts)) connect(); ASTPtr input_function; - const auto * insert = parsed_query->as(); if (insert && insert->select) insert->tryFindInputFunction(input_function); diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 2ae35c4313b..7e68fc5c4c1 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -9,7 +9,6 @@ #include #include - namespace DB { @@ -46,7 +45,9 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel // It is flattened later, when we process UNION ALL/DISTINCT. const auto * last_select = children.back()->as(); if (last_select && last_select->settings()) - InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(/* ignore_setting_constraints= */ false); + { + InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(); + } } void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_) @@ -54,20 +55,10 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (!ast) return; - /// First apply the outermost settings. Then they could be overridden by deeper settings. - if (const auto * query_with_output = dynamic_cast(ast.get())) - { - if (query_with_output->settings_ast) - InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); - - if (const auto * create_query = ast->as(); create_query && create_query->select) - applySettingsFromSelectWithUnion(create_query->select->as(), context_); - } - if (const auto * select_query = ast->as()) { if (auto new_settings = select_query->settings()) - InterpreterSetQuery(new_settings, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); + InterpreterSetQuery(new_settings, context_).executeForCurrentContext(); } else if (const auto * select_with_union_query = ast->as()) { @@ -76,15 +67,28 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta else if (const auto * explain_query = ast->as()) { if (explain_query->settings_ast) - InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(); applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } + else if (const auto * query_with_output = dynamic_cast(ast.get())) + { + if (query_with_output->settings_ast) + InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(); + + if (const auto * create_query = ast->as()) + { + if (create_query->select) + { + applySettingsFromSelectWithUnion(create_query->select->as(), context_); + } + } + } else if (auto * insert_query = ast->as()) { context_->setInsertFormat(insert_query->format); if (insert_query->settings_ast) - InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); + InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(); } } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index f50105c39f4..2438762f347 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(bool ignore_setting_constraints); + void executeForCurrentContext(bool ignore_setting_constraints = false); bool supportsTransactions() const override { return true; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index ac8f7d560e0..cb0c10cd1c9 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -151,55 +152,37 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } - /// These two sections are allowed in an arbitrary order. ParserKeyword s_format(Keyword::FORMAT); - ParserKeyword s_settings(Keyword::SETTINGS); - /** Why: let's take the following example: - * SELECT 1 UNION ALL SELECT 2 FORMAT TSV - * Each subquery can be put in parentheses and have its own settings: - * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV - * And the whole query can have settings: - * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV SETTINGS e=f - * A single query with output is parsed in the same way as the UNION ALL chain: - * SELECT 1 SETTINGS a=b FORMAT TSV SETTINGS e=f - * So while these forms have a slightly different meaning, they both exist: - * SELECT 1 SETTINGS a=b FORMAT TSV - * SELECT 1 FORMAT TSV SETTINGS e=f - * And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. - * But while this work: - * (SELECT 1) UNION ALL (SELECT 2) FORMAT TSV SETTINGS d=f - * This does not work automatically, unless we explicitly allow different orders: - * (SELECT 1) UNION ALL (SELECT 2) SETTINGS d=f FORMAT TSV - * Inevitably, we also allow this: - * SELECT 1 SETTINGS a=b SETTINGS d=f FORMAT TSV - * ^^^^^^^^^^^^^^^^^^^^^ - * Because this part is consumed into ASTSelectWithUnionQuery - * and the rest into ASTQueryWithOutput. - */ - - for (size_t i = 0; i < 2; ++i) + if (s_format.ignore(pos, expected)) { - if (!query_with_output.format && s_format.ignore(pos, expected)) - { - ParserIdentifier format_p; + ParserIdentifier format_p; - if (!format_p.parse(pos, query_with_output.format, expected)) - return false; - setIdentifierSpecial(query_with_output.format); + if (!format_p.parse(pos, query_with_output.format, expected)) + return false; + setIdentifierSpecial(query_with_output.format); - query_with_output.children.push_back(query_with_output.format); - } - else if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) + query_with_output.children.push_back(query_with_output.format); + } + + // SETTINGS key1 = value1, key2 = value2, ... + ParserKeyword s_settings(Keyword::SETTINGS); + if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) + { + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) + return false; + query_with_output.children.push_back(query_with_output.settings_ast); + + // SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery) + // Pass them manually, to apply in InterpreterSelectQuery::initSettings() + if (query->as()) { - // SETTINGS key1 = value1, key2 = value2, ... - ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) - return false; - query_with_output.children.push_back(query_with_output.settings_ast); + auto settings = query_with_output.settings_ast->clone(); + assert_cast(settings.get())->print_in_format = false; + QueryWithOutputSettingsPushDownVisitor::Data data{settings}; + QueryWithOutputSettingsPushDownVisitor(data).visit(query); } - else - break; } node = std::move(query); diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp new file mode 100644 index 00000000000..8cf0d0063ae --- /dev/null +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +bool QueryWithOutputSettingsPushDownMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) +{ + if (node->as()) + return true; + if (node->as()) + return true; + if (child->as()) + return true; + return false; +} + +void QueryWithOutputSettingsPushDownMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * select_query = ast->as()) + visit(*select_query, ast, data); +} + +void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query, ASTPtr &, Data & data) +{ + ASTPtr select_settings_ast = select_query.settings(); + if (!select_settings_ast) + { + select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, data.settings_ast->clone()); + return; + } + + SettingsChanges & select_settings = select_settings_ast->as().changes; + SettingsChanges & settings = data.settings_ast->as().changes; + + for (auto & setting : settings) + { + auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) + { + return select_setting.name == setting.name; + }); + if (it == select_settings.end()) + select_settings.push_back(setting); + else + it->value = setting.value; + } +} + +} diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h new file mode 100644 index 00000000000..fde8a07b555 --- /dev/null +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTSelectQuery; +struct SettingChange; +class SettingsChanges; + +/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: +/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput) +/// +/// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 -> +/// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1 +/// +/// Otherwise settings after FORMAT will not be applied. +class QueryWithOutputSettingsPushDownMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + const ASTPtr & settings_ast; + }; + + static bool needChildVisit(ASTPtr & node, const ASTPtr & child); + static void visit(ASTPtr & ast, Data & data); + +private: + static void visit(ASTSelectQuery &, ASTPtr &, Data &); +}; + +using QueryWithOutputSettingsPushDownVisitor = QueryWithOutputSettingsPushDownMatcher::Visitor; + +} diff --git a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql index 092b071cb48..2044a9b8d22 100644 --- a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql +++ b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql @@ -1,3 +1,4 @@ + DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS other_table; diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference index a8b99666654..22405bf1866 100644 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference @@ -1,7 +1,7 @@ 1 1 1 +1 +1 2 -1 -2 2 diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh index 173cc949500..b70c28422c9 100755 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) FORMAT CSV SETTINGS max_block_size = 1' # push down append ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_compress_block_size = 1 FORMAT CSV SETTINGS max_block_size = 1' -# not overwrite on push down +# overwrite on push down (since these settings goes latest) ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 2 FORMAT CSV SETTINGS max_block_size = 1' # on push-down ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 1 FORMAT CSV' diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql index 3a09eec7452..48e98798c51 100644 --- a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -1 +1,2 @@ -select 42 settings compatibility=NULL; -- {clientError BAD_GET} +select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS} + diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.reference b/tests/queries/0_stateless/03172_format_settings_clauses.reference deleted file mode 100644 index 8a98b137f4b..00000000000 --- a/tests/queries/0_stateless/03172_format_settings_clauses.reference +++ /dev/null @@ -1,14 +0,0 @@ -1 -2 -1 -2 -1 -2 -1 -1 -3 -3 -3 -3 -3 -1 diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.sql b/tests/queries/0_stateless/03172_format_settings_clauses.sql deleted file mode 100644 index 0d1aa4dcfbb..00000000000 --- a/tests/queries/0_stateless/03172_format_settings_clauses.sql +++ /dev/null @@ -1,30 +0,0 @@ -SET max_block_size = 10, max_threads = 1; - --- Take the following example: -SELECT 1 UNION ALL SELECT 2 FORMAT TSV; - --- Each subquery can be put in parentheses and have its own settings: -(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV; - --- And the whole query can have settings: -(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV SETTINGS max_block_size = 3; - --- A single query with output is parsed in the same way as the UNION ALL chain: -SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV SETTINGS max_block_size = 3; - --- So while these forms have a slightly different meaning, they both exist: -SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV; -SELECT getSetting('max_block_size') FORMAT TSV SETTINGS max_block_size = 3; - --- And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. --- But while this work: -(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) FORMAT TSV SETTINGS max_block_size = 3; - --- This does not work automatically, unless we explicitly allow different orders: -(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) SETTINGS max_block_size = 3 FORMAT TSV; - --- Inevitably, we allow this: -SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 SETTINGS max_block_size = 3 FORMAT TSV; -/*^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^*/ --- Because this part is consumed into ASTSelectWithUnionQuery --- and the rest into ASTQueryWithOutput. From 8b31388db5d88bbefee66d7bf4672945ce5ba4fe Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Tue, 20 Aug 2024 15:33:34 +0200 Subject: [PATCH 181/409] Update README.md add Guangzho meetup, fix release call --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 2120a4d1211..bd9b8f66ad0 100644 --- a/README.md +++ b/README.md @@ -34,13 +34,13 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 29 +* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 20 ## Upcoming Events Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* MORE COMING SOON! +* [ClickHouse Guangzho User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 4f799467ecaf1bf10e39f7da59aa459ab1903810 Mon Sep 17 00:00:00 2001 From: Michal Tabaszewski Date: Fri, 16 Aug 2024 22:54:09 +0200 Subject: [PATCH 182/409] Added removing query cache by tag. --- src/Common/CacheBase.h | 6 ++++++ src/Common/ICachePolicy.h | 1 + src/Common/LRUCachePolicy.h | 17 +++++++++++++++++ src/Common/SLRUCachePolicy.h | 21 +++++++++++++++++++++ src/Common/TTLCachePolicy.h | 17 +++++++++++++++++ src/Interpreters/Cache/QueryCache.cpp | 9 +++++++++ src/Interpreters/Cache/QueryCache.h | 1 + src/Interpreters/Context.cpp | 8 ++++++++ src/Interpreters/Context.h | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 7 +++++-- src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserSystemQuery.cpp | 10 ++++++++++ 13 files changed, 98 insertions(+), 2 deletions(-) diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index a809136f451..e0f7ffbfd5f 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -197,6 +197,12 @@ public: cache_policy->remove(key); } + void removeWithPredicate(std::function predicate) + { + std::lock_guard lock(mutex); + cache_policy->removeWithPredicate(predicate); + } + size_t sizeInBytes() const { std::lock_guard lock(mutex); diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 301a5c6cbbd..71543e0aed0 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -55,6 +55,7 @@ public: virtual void set(const Key & key, const MappedPtr & mapped) = 0; virtual void remove(const Key & key) = 0; + virtual void removeWithPredicate(std::function predicate) = 0; virtual void clear() = 0; virtual std::vector dump() const = 0; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index f833e46a821..39bf5067aea 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -68,6 +68,23 @@ public: current_size_in_bytes = 0; } + void removeWithPredicate(std::function predicate) override + { + for(auto it = cells.begin(); it != cells.end();) + { + if(predicate(it->first, it->second.value)) + { + auto & cell = it->second; + current_size_in_bytes -= cell.size; + queue.erase(cell.queue_iterator); + cells.erase(it); + it = cells.erase(it); + } + else + ++it; + } + } + void remove(const Key & key) override { auto it = cells.find(key); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 354ec1d36d6..62c6d72a3e8 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -95,6 +95,27 @@ public: cells.erase(it); } + void removeWithPredicate(std::function predicate) override + { + for(auto it = cells.begin(); it != cells.end();) + { + if(predicate(it->first, it->second.value)) + { + auto & cell = it->second; + + current_size_in_bytes -= cell.size; + if (cell.is_protected) + current_protected_size -= cell.size; + + auto & queue = cell.is_protected ? protected_queue : probationary_queue; + queue.erase(cell.queue_iterator); + it = cells.erase(it); + } + else + ++it; + } + } + MappedPtr get(const Key & key) override { auto it = cells.find(key); diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h index 6401835b0d7..6caab1d6c78 100644 --- a/src/Common/TTLCachePolicy.h +++ b/src/Common/TTLCachePolicy.h @@ -133,6 +133,23 @@ public: Base::user_quotas->clear(); } + void removeWithPredicate(std::function predicate) override + { + for(auto it = cache.begin(); it != cache.end();) + { + if(predicate(it->first, it->second)) + { + size_t sz = weight_function(*it->second); + if (it->first.user_id.has_value()) + Base::user_quotas->decreaseActual(*it->first.user_id, sz); + it = cache.erase(it); + size_in_bytes -= sz; + } + else + ++it; + } + } + void remove(const Key & key) override { auto it = cache.find(key); diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index ab926037c67..9c4d48fbd44 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -619,6 +619,15 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size); } +void QueryCache::clearWithTag(const String & tag) +{ + auto removeWithTag = [tag](const Key & k, const Cache::MappedPtr & _){ + return k.tag == tag; + }; + cache.removeWithPredicate(removeWithTag); + std::lock_guard lock(mutex); +} + void QueryCache::clear() { cache.clear(); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c7ebaf4d26a..dec43747e0a 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -212,6 +212,7 @@ public: Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size, size_t max_query_cache_size_in_bytes_quota, size_t max_query_cache_entries_quota); void clear(); + void clearWithTag(const String & tag); size_t sizeInBytes() const; size_t count() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3cc09370e86..ef5539f0b13 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3236,6 +3236,14 @@ void Context::clearQueryCache() const shared->query_cache->clear(); } +void Context::clearQueryCacheWithTag(const String & tag) const +{ + std::lock_guard lock(shared->mutex); + + if (shared->query_cache) + shared->query_cache->clearWithTag(tag); +} + void Context::clearCaches() const { std::lock_guard lock(shared->mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3da4f124553..076bff462f9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1069,6 +1069,7 @@ public: void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; void clearQueryCache() const; + void clearQueryCacheWithTag(const String & tag) const; /** Clear the caches of the uncompressed blocks and marks. * This is usually done when renaming tables, changing the type of columns, deleting a table. diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 21c8b44b374..c71982d0934 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -369,9 +369,12 @@ BlockIO InterpreterSystemQuery::execute() system_context->clearMMappedFileCache(); break; case Type::DROP_QUERY_CACHE: - getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); - getContext()->clearQueryCache(); + { + getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); + !query.query_cache_tag.empty() ? getContext()->clearQueryCacheWithTag(query.query_cache_tag): getContext()->clearQueryCache(); break; + } + case Type::DROP_COMPILED_EXPRESSION_CACHE: #if USE_EMBEDDED_COMPILER getContext()->checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 59de90b1d8e..739d71678df 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -129,6 +129,7 @@ public: String storage_policy; String volume; String disk; + String query_cache_tag; UInt64 seconds{}; String filesystem_cache_name; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index cc2ee79cd36..a079fba5d53 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -471,6 +471,7 @@ namespace DB MR_MACROS(TABLE, "TABLE") \ MR_MACROS(TABLES, "TABLES") \ MR_MACROS(TAGS, "TAGS") \ + MR_MACROS(TAG, "TAG") \ MR_MACROS(TAGS_INNER_UUID, "TAGS INNER UUID") \ MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \ MR_MACROS(TEMPORARY, "TEMPORARY") \ diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index efabbbfa479..95d89e6fde1 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -489,6 +489,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; break; } + case Type::DROP_QUERY_CACHE: + { + ParserLiteral tag_parser; + ASTPtr ast; + if (ParserKeyword{Keyword::TAG}.ignore(pos, expected) && tag_parser.parse(pos, ast, expected)) + res->query_cache_tag = ast->as()->value.safeGet(); + if (!parseQueryWithOnCluster(res, pos, expected)) + return false; + break; + } case Type::SYNC_FILESYSTEM_CACHE: { ParserLiteral path_parser; From cd51535a67319d06fae7ca212123c9cb9114c91c Mon Sep 17 00:00:00 2001 From: Michal Tabaszewski Date: Sat, 17 Aug 2024 01:54:46 +0200 Subject: [PATCH 183/409] Fixed style. --- src/Common/LRUCachePolicy.h | 4 ++-- src/Common/SLRUCachePolicy.h | 4 ++-- src/Common/TTLCachePolicy.h | 4 ++-- src/Interpreters/Cache/QueryCache.cpp | 5 +++-- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 39bf5067aea..9f2ea548c32 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -70,9 +70,9 @@ public: void removeWithPredicate(std::function predicate) override { - for(auto it = cells.begin(); it != cells.end();) + for (auto it = cells.begin(); it != cells.end();) { - if(predicate(it->first, it->second.value)) + if (predicate(it->first, it->second.value)) { auto & cell = it->second; current_size_in_bytes -= cell.size; diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 62c6d72a3e8..e6931bbd59c 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -97,9 +97,9 @@ public: void removeWithPredicate(std::function predicate) override { - for(auto it = cells.begin(); it != cells.end();) + for (auto it = cells.begin(); it != cells.end();) { - if(predicate(it->first, it->second.value)) + if (predicate(it->first, it->second.value)) { auto & cell = it->second; diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h index 6caab1d6c78..100d753e53b 100644 --- a/src/Common/TTLCachePolicy.h +++ b/src/Common/TTLCachePolicy.h @@ -135,9 +135,9 @@ public: void removeWithPredicate(std::function predicate) override { - for(auto it = cache.begin(); it != cache.end();) + for (auto it = cache.begin(); it != cache.end();) { - if(predicate(it->first, it->second)) + if (predicate(it->first, it->second)) { size_t sz = weight_function(*it->second); if (it->first.user_id.has_value()) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 9c4d48fbd44..84aa9cd0342 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -621,10 +621,11 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis void QueryCache::clearWithTag(const String & tag) { - auto removeWithTag = [tag](const Key & k, const Cache::MappedPtr & _){ + auto remove_with_tag = [tag](const Key & k, const Cache::MappedPtr & _) + { return k.tag == tag; }; - cache.removeWithPredicate(removeWithTag); + cache.removeWithPredicate(remove_with_tag); std::lock_guard lock(mutex); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c71982d0934..c7540084451 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -370,7 +370,7 @@ BlockIO InterpreterSystemQuery::execute() break; case Type::DROP_QUERY_CACHE: { - getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); + getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); !query.query_cache_tag.empty() ? getContext()->clearQueryCacheWithTag(query.query_cache_tag): getContext()->clearQueryCache(); break; } From ad20d24c6e0aa50ea408627389acb4c83080d3e0 Mon Sep 17 00:00:00 2001 From: Michal Tabaszewski Date: Mon, 19 Aug 2024 20:52:51 +0200 Subject: [PATCH 184/409] Fixed binary-tidy complaint --- src/Interpreters/Cache/QueryCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 84aa9cd0342..6c1ffe9cecb 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -621,7 +621,7 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis void QueryCache::clearWithTag(const String & tag) { - auto remove_with_tag = [tag](const Key & k, const Cache::MappedPtr & _) + auto remove_with_tag = [tag](const Key & k, const Cache::MappedPtr &) { return k.tag == tag; }; From eee618b3196cecb44f6314a4199d5aa95ff2ca66 Mon Sep 17 00:00:00 2001 From: Michal Tabaszewski Date: Mon, 19 Aug 2024 22:37:28 +0200 Subject: [PATCH 185/409] - From 129d1e387e1b436ac572ca5db1a50dd9ed79ddae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 20 Aug 2024 13:21:42 +0000 Subject: [PATCH 186/409] Some fixups --- docs/en/operations/query-cache.md | 2 ++ docs/en/sql-reference/statements/system.md | 6 ++++ src/Common/CacheBase.h | 4 +-- src/Common/ICachePolicy.h | 2 +- src/Common/LRUCachePolicy.h | 33 +++++++++---------- src/Common/SLRUCachePolicy.h | 2 +- src/Common/TTLCachePolicy.h | 26 +++++++-------- src/Interpreters/Cache/QueryCache.cpp | 19 +++++------ src/Interpreters/Cache/QueryCache.h | 3 +- src/Interpreters/Context.cpp | 12 ++----- src/Interpreters/Context.h | 3 +- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Parsers/ASTSystemQuery.h | 3 +- src/Parsers/CommonParsers.h | 2 +- src/Parsers/ParserSystemQuery.cpp | 20 +++++------ .../02494_query_cache_drop_cache.reference | 14 ++++++++ .../02494_query_cache_drop_cache.sql | 25 ++++++++++++-- 17 files changed, 105 insertions(+), 73 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 384938e28f6..955cec0234e 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -155,6 +155,8 @@ SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 1'; SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 2'; ``` +To remove only entries with tag `tag` from the query cache, you can use statement `SYSTEM DROP QUERY CACHE TAG 'tag'`. + ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 3ebcf617491..77d023b67ce 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -136,7 +136,13 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le ## DROP QUERY CACHE +```sql +SYSTEM DROP QUERY CACHE; +SYSTEM DROP QUERY CACHE TAG '' +```` + Clears the [query cache](../../operations/query-cache.md). +If a tag is specified, only query cache entries with the specified tag are deleted. ## DROP FORMAT SCHEMA CACHE {#system-drop-schema-format} diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index e0f7ffbfd5f..23e6a6fc91c 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -197,10 +197,10 @@ public: cache_policy->remove(key); } - void removeWithPredicate(std::function predicate) + void remove(std::function predicate) { std::lock_guard lock(mutex); - cache_policy->removeWithPredicate(predicate); + cache_policy->remove(predicate); } size_t sizeInBytes() const diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 71543e0aed0..567fa35d977 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -55,7 +55,7 @@ public: virtual void set(const Key & key, const MappedPtr & mapped) = 0; virtual void remove(const Key & key) = 0; - virtual void removeWithPredicate(std::function predicate) = 0; + virtual void remove(std::function predicate) = 0; virtual void clear() = 0; virtual std::vector dump() const = 0; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 9f2ea548c32..cb8fdbd2b9c 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -68,23 +68,6 @@ public: current_size_in_bytes = 0; } - void removeWithPredicate(std::function predicate) override - { - for (auto it = cells.begin(); it != cells.end();) - { - if (predicate(it->first, it->second.value)) - { - auto & cell = it->second; - current_size_in_bytes -= cell.size; - queue.erase(cell.queue_iterator); - cells.erase(it); - it = cells.erase(it); - } - else - ++it; - } - } - void remove(const Key & key) override { auto it = cells.find(key); @@ -96,6 +79,22 @@ public: cells.erase(it); } + void remove(std::function predicate) override + { + for (auto it = cells.begin(); it != cells.end();) + { + if (predicate(it->first, it->second.value)) + { + Cell & cell = it->second; + current_size_in_bytes -= cell.size; + queue.erase(cell.queue_iterator); + it = cells.erase(it); + } + else + ++it; + } + } + MappedPtr get(const Key & key) override { auto it = cells.find(key); diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index e6931bbd59c..5321110f3e5 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -95,7 +95,7 @@ public: cells.erase(it); } - void removeWithPredicate(std::function predicate) override + void remove(std::function predicate) override { for (auto it = cells.begin(); it != cells.end();) { diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h index 100d753e53b..6c548e5042b 100644 --- a/src/Common/TTLCachePolicy.h +++ b/src/Common/TTLCachePolicy.h @@ -133,7 +133,19 @@ public: Base::user_quotas->clear(); } - void removeWithPredicate(std::function predicate) override + void remove(const Key & key) override + { + auto it = cache.find(key); + if (it == cache.end()) + return; + size_t sz = weight_function(*it->second); + if (it->first.user_id.has_value()) + Base::user_quotas->decreaseActual(*it->first.user_id, sz); + cache.erase(it); + size_in_bytes -= sz; + } + + void remove(std::function predicate) override { for (auto it = cache.begin(); it != cache.end();) { @@ -150,18 +162,6 @@ public: } } - void remove(const Key & key) override - { - auto it = cache.find(key); - if (it == cache.end()) - return; - size_t sz = weight_function(*it->second); - if (it->first.user_id.has_value()) - Base::user_quotas->decreaseActual(*it->first.user_id, sz); - cache.erase(it); - size_in_bytes -= sz; - } - MappedPtr get(const Key & key) override { auto it = cache.find(key); diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 6c1ffe9cecb..4312b35e18c 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -619,19 +619,18 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size); } -void QueryCache::clearWithTag(const String & tag) +void QueryCache::clear(const std::optional & tag) { - auto remove_with_tag = [tag](const Key & k, const Cache::MappedPtr &) + if (tag) { - return k.tag == tag; - }; - cache.removeWithPredicate(remove_with_tag); - std::lock_guard lock(mutex); -} + auto predicate = [tag](const Key & key, const Cache::MappedPtr &) { return key.tag == tag.value(); }; + cache.remove(predicate); + } + else + { + cache.clear(); + } -void QueryCache::clear() -{ - cache.clear(); std::lock_guard lock(mutex); times_executed.clear(); } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index dec43747e0a..64407633a8d 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -211,8 +211,7 @@ public: Reader createReader(const Key & key); Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size, size_t max_query_cache_size_in_bytes_quota, size_t max_query_cache_entries_quota); - void clear(); - void clearWithTag(const String & tag); + void clear(const std::optional & tag); size_t sizeInBytes() const; size_t count() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ef5539f0b13..cfcaf437510 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3228,20 +3228,12 @@ QueryCachePtr Context::getQueryCache() const return shared->query_cache; } -void Context::clearQueryCache() const +void Context::clearQueryCache(const std::optional & tag) const { std::lock_guard lock(shared->mutex); if (shared->query_cache) - shared->query_cache->clear(); -} - -void Context::clearQueryCacheWithTag(const String & tag) const -{ - std::lock_guard lock(shared->mutex); - - if (shared->query_cache) - shared->query_cache->clearWithTag(tag); + shared->query_cache->clear(tag); } void Context::clearCaches() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 076bff462f9..e0c69471e60 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1068,8 +1068,7 @@ public: void setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows); void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; - void clearQueryCache() const; - void clearQueryCacheWithTag(const String & tag) const; + void clearQueryCache(const std::optional & tag) const; /** Clear the caches of the uncompressed blocks and marks. * This is usually done when renaming tables, changing the type of columns, deleting a table. diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c7540084451..d4e2f22036c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -371,7 +371,7 @@ BlockIO InterpreterSystemQuery::execute() case Type::DROP_QUERY_CACHE: { getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); - !query.query_cache_tag.empty() ? getContext()->clearQueryCacheWithTag(query.query_cache_tag): getContext()->clearQueryCache(); + getContext()->clearQueryCache(query.query_cache_tag); break; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 739d71678df..d9f5b425182 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -129,9 +129,10 @@ public: String storage_policy; String volume; String disk; - String query_cache_tag; UInt64 seconds{}; + std::optional query_cache_tag; + String filesystem_cache_name; std::string key_to_drop; std::optional offset_to_drop; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index a079fba5d53..ab0e70eb0e5 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -470,8 +470,8 @@ namespace DB MR_MACROS(TABLE_OVERRIDE, "TABLE OVERRIDE") \ MR_MACROS(TABLE, "TABLE") \ MR_MACROS(TABLES, "TABLES") \ - MR_MACROS(TAGS, "TAGS") \ MR_MACROS(TAG, "TAG") \ + MR_MACROS(TAGS, "TAGS") \ MR_MACROS(TAGS_INNER_UUID, "TAGS INNER UUID") \ MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \ MR_MACROS(TEMPORARY, "TEMPORARY") \ diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 95d89e6fde1..af84dd10bfa 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -471,6 +471,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & res->seconds = seconds->as()->value.safeGet(); break; } + case Type::DROP_QUERY_CACHE: + { + ParserLiteral tag_parser; + ASTPtr ast; + if (ParserKeyword{Keyword::TAG}.ignore(pos, expected) && tag_parser.parse(pos, ast, expected)) + res->query_cache_tag = std::make_optional(ast->as()->value.safeGet()); + if (!parseQueryWithOnCluster(res, pos, expected)) + return false; + break; + } case Type::DROP_FILESYSTEM_CACHE: { ParserLiteral path_parser; @@ -489,16 +499,6 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; break; } - case Type::DROP_QUERY_CACHE: - { - ParserLiteral tag_parser; - ASTPtr ast; - if (ParserKeyword{Keyword::TAG}.ignore(pos, expected) && tag_parser.parse(pos, ast, expected)) - res->query_cache_tag = ast->as()->value.safeGet(); - if (!parseQueryWithOnCluster(res, pos, expected)) - return false; - break; - } case Type::SYNC_FILESYSTEM_CACHE: { ParserLiteral path_parser; diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.reference b/tests/queries/0_stateless/02494_query_cache_drop_cache.reference index 2f1465d1598..6481b5e0770 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.reference +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.reference @@ -1,3 +1,17 @@ +Cache query result in query cache 1 1 +DROP entries with a certain tag, no entry will match +1 +After a full DROP, the cache is empty now +0 +Cache query result with different or no tag in query cache +1 +1 +1 +2 +4 +DROP entries with certain tags +2 +1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql index bc2e7f442fc..3d064169a4e 100644 --- a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql +++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql @@ -4,10 +4,31 @@ -- (it's silly to use what will be tested below but we have to assume other tests cluttered the query cache) SYSTEM DROP QUERY CACHE; --- Cache query result in query cache +SELECT 'Cache query result in query cache'; SELECT 1 SETTINGS use_query_cache = true; SELECT count(*) FROM system.query_cache; --- No query results are cached after DROP +SELECT 'DROP entries with a certain tag, no entry will match'; +SYSTEM DROP QUERY CACHE TAG 'tag'; +SELECT count(*) FROM system.query_cache; + +SELECT 'After a full DROP, the cache is empty now'; SYSTEM DROP QUERY CACHE; SELECT count(*) FROM system.query_cache; + +-- More tests for DROP with tags: + +SELECT 'Cache query result with different or no tag in query cache'; +SELECT 1 SETTINGS use_query_cache = true; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'def'; +SELECT 2 SETTINGS use_query_cache = true; +SELECT count(*) FROM system.query_cache; + +SELECT 'DROP entries with certain tags'; +SYSTEM DROP QUERY CACHE TAG ''; +SELECT count(*) FROM system.query_cache; +SYSTEM DROP QUERY CACHE TAG 'def'; +SELECT count(*) FROM system.query_cache; +SYSTEM DROP QUERY CACHE TAG 'abc'; +SELECT count(*) FROM system.query_cache; From f5a020247486442f74861c61f162c647ebd97f8d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 20 Aug 2024 10:35:51 +0000 Subject: [PATCH 187/409] Add integration test --- .../NamedCollectionsMetadataStorage.cpp | 4 +- .../__init__.py | 0 .../config.d/named_collections_encrypted.xml | 12 ++ ...d_collections_with_zookeeper_encrypted.xml | 31 +++++ .../configs/users.d/users.xml | 17 +++ .../test_named_collections_encrypted/test.py | 115 ++++++++++++++++++ 6 files changed, 177 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_named_collections_encrypted/__init__.py create mode 100644 tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml create mode 100644 tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml create mode 100644 tests/integration/test_named_collections_encrypted/configs/users.d/users.xml create mode 100644 tests/integration/test_named_collections_encrypted/test.py diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index b8269d2d55a..8a6f3b9f6d5 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -628,9 +628,9 @@ std::unique_ptr NamedCollectionsMetadataStorage const auto path = config.getString(named_collections_storage_config_path + ".path"); std::unique_ptr zk_storage; - if (storage_type == "zookeeper" || storage_type == "keeper") + if (!storage_type.ends_with("_encrypted")) zk_storage = std::make_unique(context_, path); - else if (storage_type == "zookeeper_encrypted" || storage_type == "keeper_encrypted") + else zk_storage = std::make_unique(context_, path); LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), diff --git a/tests/integration/test_named_collections_encrypted/__init__.py b/tests/integration/test_named_collections_encrypted/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml new file mode 100644 index 00000000000..233e23846cb --- /dev/null +++ b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml @@ -0,0 +1,12 @@ + + + local_encrypted + bebec0cabebec0cabebec0cabebec0ca + + + + + value1 + + + diff --git a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml new file mode 100644 index 00000000000..d1dd5c29787 --- /dev/null +++ b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml @@ -0,0 +1,31 @@ + + + zookeeper_encrypted + bebec0cabebec0cabebec0cabebec0ca + /named_collections_path/ + 5000 + + + + + value1 + + + + + + + true + + node_with_keeper + 9000 + + + node_with_keeper_2 + 9000 + + + true + + + diff --git a/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml b/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml new file mode 100644 index 00000000000..7d4f0543ff1 --- /dev/null +++ b/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml @@ -0,0 +1,17 @@ + + + + 0 + + + + + + default + default + 1 + 1 + 1 + + + diff --git a/tests/integration/test_named_collections_encrypted/test.py b/tests/integration/test_named_collections_encrypted/test.py new file mode 100644 index 00000000000..adc60b08b7f --- /dev/null +++ b/tests/integration/test_named_collections_encrypted/test.py @@ -0,0 +1,115 @@ +import logging +import pytest +import os +from helpers.cluster import ClickHouseCluster + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +NAMED_COLLECTIONS_CONFIG = os.path.join( + SCRIPT_DIR, "./configs/config.d/named_collections.xml" +) + +ZK_PATH = "/named_collections_path" + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node_encrypted", + main_configs=[ + "configs/config.d/named_collections_encrypted.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + ) + cluster.add_instance( + "node_with_keeper_encrypted", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper_encrypted.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "node_with_keeper_2_encrypted", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper_encrypted.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def check_encrypted_content(node, zk=None): + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections").strip() + ) + + assert ( + "['key1','key2']" + == node.query( + "select mapKeys(collection) from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "1234\tvalue2" + == node.query( + "select collection['key1'], collection['key2'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + # Check that the underlying storage is encrypted + content = zk.get(ZK_PATH + "/collection2.sql")[0] if zk is not None else open(f"{node.path}/database/named_collections/collection2.sql", "rb").read() + + assert content[0:3] == b"ENC" # file signature (aka magic number) of the encrypted file + assert b"key1" not in content + assert b"1234" not in content + assert b"key2" not in content + assert B"value2" not in content + + +def test_local_storage_encrypted(cluster): + node = cluster.instances["node_encrypted"] + node.query("CREATE NAMED COLLECTION collection2 AS key1=1234, key2='value2'") + + check_encrypted_content(node) + node.restart_clickhouse() + check_encrypted_content(node) + + node.query("DROP NAMED COLLECTION collection2") + + +def test_zookeper_storage_encrypted(cluster): + node1 = cluster.instances["node_with_keeper_encrypted"] + node2 = cluster.instances["node_with_keeper_2_encrypted"] + zk = cluster.get_kazoo_client("zoo1") + + node1.query("CREATE NAMED COLLECTION collection2 AS key1=1234, key2='value2'") + + check_encrypted_content(node1, zk) + check_encrypted_content(node2, zk) + node1.restart_clickhouse() + node2.restart_clickhouse() + check_encrypted_content(node1, zk) + check_encrypted_content(node2, zk) + + node1.query("DROP NAMED COLLECTION collection2") From e416a2b3d2f9ff2395a218e79f9417cb96dafbda Mon Sep 17 00:00:00 2001 From: leonkozlowski Date: Tue, 20 Aug 2024 09:42:19 -0400 Subject: [PATCH 188/409] patch: fix reference to sorting key in primary key docs --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 183b94f4641..0b693775dde 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -80,7 +80,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key. -It is usually not necessary to specify the primary key in addition to the primary key. +It is usually not necessary to specify the primary key in addition to the sorting key. #### SAMPLE BY From d10e65291ccea45e46aab5edcc21ee047de6d5fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 2 Jul 2024 10:32:51 +0000 Subject: [PATCH 189/409] Fix docs --- docs/en/operations/settings/settings.md | 12 +++++++++++- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 19db4be17db..3617e6a3167 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2855,7 +2855,7 @@ The minimum chunk size in bytes, which each thread will parse in parallel. ## merge_selecting_sleep_ms {#merge_selecting_sleep_ms} -Sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. +Minimum sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. Possible values: @@ -2863,6 +2863,16 @@ Possible values: Default value: `5000`. +## max_merge_selecting_sleep_ms + +Maximum sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. + +Possible values: + +- Any positive integer. + +Default value: `60000`. + ## parallel_distributed_insert_select {#parallel_distributed_insert_select} Enables parallel distributed `INSERT ... SELECT` query. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index de1f0f60cfc..676f776df54 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -67,7 +67,7 @@ struct Settings; M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ - M(UInt64, merge_selecting_sleep_ms, 5000, "Maximum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(UInt64, merge_selecting_sleep_ms, 5000, "Minimum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ M(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ M(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ From e547875a9cdffc53bb601c1f9d32c2a6fb6af560 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 5 Jul 2024 13:27:16 +0000 Subject: [PATCH 190/409] Improve wording --- docs/en/operations/settings/settings.md | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3617e6a3167..7bd36ccd00f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2855,7 +2855,7 @@ The minimum chunk size in bytes, which each thread will parse in parallel. ## merge_selecting_sleep_ms {#merge_selecting_sleep_ms} -Minimum sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. +Minimum time to wait before trying to select parts to merge again after no parts were selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. Possible values: @@ -2865,7 +2865,7 @@ Default value: `5000`. ## max_merge_selecting_sleep_ms -Maximum sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. +Maximum time to wait before trying to select parts to merge again after no parts were selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters. Possible values: diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 676f776df54..0769b60dc6b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -67,8 +67,8 @@ struct Settings; M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ - M(UInt64, merge_selecting_sleep_ms, 5000, "Minimum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ - M(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(UInt64, merge_selecting_sleep_ms, 5000, "Minimum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ M(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ From c56ae57af861a3bc7bcc1f423b22c6c3256b2b73 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 20 Aug 2024 14:06:39 +0000 Subject: [PATCH 191/409] Update documentation --- docs/en/operations/named-collections.md | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 340a5a8f87a..470acaa7200 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -73,13 +73,21 @@ In the above example the `password_sha256_hex` value is the hexadecimal represen ### Storage for named collections -Named collections can either be stored on local disk or in zookeeper/keeper. By default local storage is used. +Named collections can either be stored on local disk or in ZooKeeper/Keeper. By default local storage is used. +They can also be stored using encryption with the same algorithms used for [disk encryption](storing-data#encrypted-virtual-file-system), +where `aes_128_ctr` is used by default. -To configure named collections storage in keeper and a `type` (equal to either `keeper` or `zookeeper`) and `path` (path in keeper, where named collections will be stored) to `named_collections_storage` section in configuration file: +To configure named collections storage you need to speficy a `type`. This can be either `local` or `keeper`/`zookeeper`. For encrypted storage, +you can use `local_encrypted` or `keeper_encrypted`/`zookeeper_encrypted`. + +To use ZooKeeper/Keeper we also need to set up a `path` (path in ZooKeeper/Keeper, where named collections will be stored) to +`named_collections_storage` section in configuration file. The following example uses encryption and ZooKeeper/Keeper: ``` - zookeeper + zookeeper_encrypted + bebec0cabebec0cabebec0cabebec0ca + aes_128_ctr /named_collections_path/ 1000 @@ -315,7 +323,7 @@ The description of parameters see [postgresql](../sql-reference/table-functions/ Parameter `addresses_expr` is used in a collection instead of `host:port`. The parameter is optional, because there are other optional ones: `host`, `hostname`, `port`. The following pseudo code explains the priority: ```sql -CASE +CASE WHEN collection['addresses_expr'] != '' THEN collection['addresses_expr'] WHEN collection['host'] != '' THEN collection['host'] || ':' || if(collection['port'] != '', collection['port'], '5432') WHEN collection['hostname'] != '' THEN collection['hostname'] || ':' || if(collection['port'] != '', collection['port'], '5432') @@ -496,7 +504,7 @@ kafka_topic_list = 'kafka_topic', kafka_group_name = 'consumer_group', kafka_format = 'JSONEachRow', kafka_max_block_size = '1048576'; - + ``` ### XML example From dd3921897195a7027ef060f166ebcb611608763c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 20 Aug 2024 14:24:53 +0000 Subject: [PATCH 192/409] Fix style --- docs/en/operations/named-collections.md | 2 +- .../test_named_collections_encrypted/test.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 470acaa7200..1c82aeaaf2c 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -77,7 +77,7 @@ Named collections can either be stored on local disk or in ZooKeeper/Keeper. By They can also be stored using encryption with the same algorithms used for [disk encryption](storing-data#encrypted-virtual-file-system), where `aes_128_ctr` is used by default. -To configure named collections storage you need to speficy a `type`. This can be either `local` or `keeper`/`zookeeper`. For encrypted storage, +To configure named collections storage you need to specify a `type`. This can be either `local` or `keeper`/`zookeeper`. For encrypted storage, you can use `local_encrypted` or `keeper_encrypted`/`zookeeper_encrypted`. To use ZooKeeper/Keeper we also need to set up a `path` (path in ZooKeeper/Keeper, where named collections will be stored) to diff --git a/tests/integration/test_named_collections_encrypted/test.py b/tests/integration/test_named_collections_encrypted/test.py index adc60b08b7f..7dff32fa6c9 100644 --- a/tests/integration/test_named_collections_encrypted/test.py +++ b/tests/integration/test_named_collections_encrypted/test.py @@ -78,13 +78,21 @@ def check_encrypted_content(node, zk=None): ) # Check that the underlying storage is encrypted - content = zk.get(ZK_PATH + "/collection2.sql")[0] if zk is not None else open(f"{node.path}/database/named_collections/collection2.sql", "rb").read() + content = ( + zk.get(ZK_PATH + "/collection2.sql")[0] + if zk is not None + else open( + f"{node.path}/database/named_collections/collection2.sql", "rb" + ).read() + ) - assert content[0:3] == b"ENC" # file signature (aka magic number) of the encrypted file + assert ( + content[0:3] == b"ENC" + ) # file signature (aka magic number) of the encrypted file assert b"key1" not in content assert b"1234" not in content assert b"key2" not in content - assert B"value2" not in content + assert b"value2" not in content def test_local_storage_encrypted(cluster): From 677b28e1ac5f0962f1e43ba7589787105c5f1553 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 20 Aug 2024 17:04:19 +0200 Subject: [PATCH 193/409] Fix docs --- docs/en/engines/table-engines/integrations/hdfs.md | 2 +- docs/en/engines/table-engines/integrations/s3.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index c9df713231a..404cec97def 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -240,7 +240,7 @@ libhdfs3 support HDFS namenode HA. ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. -- [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [hdfs_create_new_file_on_insert](/docs/en/operations/settings/settings.md#hdfs_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. - [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. **See Also** diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index d664c37bd0f..48a08dfa499 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -225,7 +225,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ## Storage Settings {#storage-settings} - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. -- [s3_create_multiple_files](/docs/en/operations/settings/settings.md#s3_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [s3_create_new_file_on_insert](/docs/en/operations/settings/settings.md#s3_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. - [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. ## S3-related Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index f96e48d914d..30d2e371c7e 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -116,7 +116,7 @@ SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parque ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. -- [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [hdfs_create_new_file_on_insert](/docs/en/operations/settings/settings.md#hdfs_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. - [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. - [ignore_access_denied_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 1bd9f38517e..181c92b92d4 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -290,7 +290,7 @@ SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date ## Storage Settings {#storage-settings} - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. -- [s3_create_multiple_files](/docs/en/operations/settings/settings.md#s3_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [s3_create_new_file_on_insert](/docs/en/operations/settings/settings.md#s3_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. - [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. **See Also** From eb2f50e49d45f9418d9490f8c3da1af23006e9cc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Aug 2024 15:28:04 +0000 Subject: [PATCH 194/409] Do not fuzz settings for 02835_drop_user_during_session.sh --- tests/queries/0_stateless/02835_drop_user_during_session.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index c32003a2a11..01e4f9a5c2b 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-debug +# Tags: no-debug, no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0091f16af9426acbcc696b57f813aeb1222e0687 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 20 Aug 2024 16:13:16 +0000 Subject: [PATCH 195/409] Fix build without SSL support --- .../NamedCollectionsMetadataStorage.cpp | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 8a6f3b9f6d5..e9f7816ce73 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int SUPPORT_IS_DISABLED; } static const std::string named_collections_storage_config_path = "named_collections_storage"; @@ -361,6 +362,8 @@ private: } }; +#if USE_SSL + template class NamedCollectionsMetadataStorageEncrypted : public BaseMetadataStorage { @@ -444,6 +447,8 @@ class NamedCollectionsMetadataStorage::ZooKeeperStorageEncrypted : public NamedC using NamedCollectionsMetadataStorageEncrypted::NamedCollectionsMetadataStorageEncrypted; }; +#endif + NamedCollectionsMetadataStorage::NamedCollectionsMetadataStorage( std::shared_ptr storage_, ContextPtr context_) @@ -618,7 +623,13 @@ std::unique_ptr NamedCollectionsMetadataStorage if (storage_type == "local") local_storage = std::make_unique(context_, path); else if (storage_type == "local_encrypted") + { +#if USE_SSL local_storage = std::make_unique(context_, path); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Named collections encryption requires building with SSL support"); +#endif + } return std::unique_ptr( new NamedCollectionsMetadataStorage(std::move(local_storage), context_)); @@ -631,7 +642,13 @@ std::unique_ptr NamedCollectionsMetadataStorage if (!storage_type.ends_with("_encrypted")) zk_storage = std::make_unique(context_, path); else + { +#if USE_SSL zk_storage = std::make_unique(context_, path); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Named collections encryption requires building with SSL support"); +#endif + } LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), "Using zookeeper storage for named collections at path: {}", path); From fe637452ec730db224e40a5c4a399d9ff7ac4ca0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 20 Aug 2024 19:54:12 +0200 Subject: [PATCH 196/409] Revert "Fix test `01079_bad_alters_zookeeper_long`" --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- tests/clickhouse-test | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 80a7e862f72..ff8e362aa36 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6340,7 +6340,7 @@ void StorageReplicatedMergeTree::alter( "Metadata on replica is not up to date with common metadata in Zookeeper. " "It means that this replica still not applied some of previous alters." " Probably too many alters executing concurrently (highly not recommended). " - "You can retry the query"); + "You can retry this error"); /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. if (query_context->getZooKeeperMetadataTransaction()) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5fb892597f7..01c2937352f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -59,7 +59,6 @@ MESSAGES_TO_RETRY = [ "is already started to be removing by another replica right now", # This is from LSan, and it indicates its own internal problem: "Unable to get registers from thread", - "You can retry", ] MAX_RETRIES = 3 From 2ad50a5f3c6a1da2e33aee32051d654e789b8ca3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 20 Aug 2024 19:56:22 +0200 Subject: [PATCH 197/409] Update 01079_bad_alters_zookeeper_long.sh --- tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh b/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh index 82b8be65af5..39e65af039b 100755 --- a/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh @@ -26,6 +26,10 @@ while [[ $($CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='00000000 sleep 1 done +while [[ $($CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE type='ALTER_METADATA' AND database = '$CLICKHOUSE_DATABASE'" 2>&1) ]]; do + sleep 1 +done + $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_bad_alters;" # Type changed, but we can revert back $CLICKHOUSE_CLIENT --query "INSERT INTO table_for_bad_alters VALUES(2, 2, 7)" From 4d0fc70227e4b850953630508f65b06cbafc62f0 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 8 Aug 2024 10:07:05 +0200 Subject: [PATCH 198/409] Fix keyed hash issue with empty array/map input --- src/Functions/FunctionsHashing.h | 5 ++--- tests/queries/0_stateless/02534_keyed_siphash.reference | 5 +++++ tests/queries/0_stateless/02534_keyed_siphash.sql | 6 ++++++ 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 95c54ac9528..3dd63b64346 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -96,9 +96,8 @@ namespace impl { const auto *const begin = offsets->begin(); const auto * upper = std::upper_bound(begin, offsets->end(), i); - if (upper == offsets->end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "offset {} not found in function SipHashKeyColumns::getKey", i); - i = upper - begin; + if (upper != offsets->end()) + i = upper - begin; } const auto & key0data = assert_cast(*key0).getData(); const auto & key1data = assert_cast(*key1).getData(); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3f478218ff1..22644ca1291 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -239,3 +239,8 @@ Check bug found fuzzing Check bug 2 found fuzzing 608E1FF030C9E206185B112C2A25F1A7 ABB65AE97711A2E053E324ED88B1D08B +Check bug 3 found fuzzing +4761183170873013810 +0AD04BFD000000000000000000000000 +4761183170873013810 +0AD04BFD000000000000000000000000 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index fb707109c83..a595a97592a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -345,3 +345,9 @@ CREATE TABLE sipHashKeyed_keys (`a` Map(String, String)) ENGINE = Memory; INSERT INTO sipHashKeyed_keys FORMAT VALUES ({'a':'b', 'c':'d'}), ({'e':'f', 'g':'h'}); SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FROM sipHashKeyed_keys ORDER BY a; DROP TABLE sipHashKeyed_keys; + +SELECT 'Check bug 3 found fuzzing'; +SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []::Array(UInt8)); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), []::Array(UInt8))); +SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); From ef2912b0f7d225a6c390ddbc77b7a30b994fe1a8 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 9 Aug 2024 18:09:52 +0200 Subject: [PATCH 199/409] Fix old and wrong assert --- src/Functions/FunctionsHashing.h | 5 +++-- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 ++ tests/queries/0_stateless/02534_keyed_siphash.sql | 2 ++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3dd63b64346..c01ad5cf7e0 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -83,8 +83,7 @@ namespace impl { assert(key0 && key1); assert(key0->size() == key1->size()); - assert(offsets == nullptr || offsets->size() == key0->size()); - if (offsets != nullptr) + if (offsets != nullptr && !offsets->empty()) return offsets->back(); return key0->size(); } @@ -92,6 +91,8 @@ namespace impl { if (is_const) i = 0; + assert(key0->size() == key1->size()); + assert(key0->size() > i); if (offsets != nullptr) { const auto *const begin = offsets->begin(); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 22644ca1291..b65992a5ce3 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -244,3 +244,5 @@ Check bug 3 found fuzzing 0AD04BFD000000000000000000000000 4761183170873013810 0AD04BFD000000000000000000000000 +16734549324845627102 +D675BB3D687973A238AB891DD99C7047 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index a595a97592a..4e6b32966d4 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -351,3 +351,5 @@ SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []::Array(UInt8)); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), []::Array(UInt8))); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); +SELECT sipHash64Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3)); +SELECT hex(sipHash128Keyed((0::UInt64, 0::UInt64), map([0], 1, [2], 3))); From 6b7a25746af94e1dbb85de65304e47e041d4b006 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 13 Aug 2024 18:05:09 +0200 Subject: [PATCH 200/409] Improve test description --- tests/queries/0_stateless/02534_keyed_siphash.reference | 4 ++-- tests/queries/0_stateless/02534_keyed_siphash.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index b65992a5ce3..31c0cae8981 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -236,10 +236,10 @@ Check asan bug 0 Check bug found fuzzing 9042C6691B1A75F0EA3314B6F55728BB -Check bug 2 found fuzzing +Test arrays and maps 608E1FF030C9E206185B112C2A25F1A7 ABB65AE97711A2E053E324ED88B1D08B -Check bug 3 found fuzzing +Test emtpy arrays and maps 4761183170873013810 0AD04BFD000000000000000000000000 4761183170873013810 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 4e6b32966d4..b96233200a8 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -339,14 +339,14 @@ SELECT 'Check bug found fuzzing'; SELECT [(255, 1048575)], sipHash128ReferenceKeyed((toUInt64(2147483646), toUInt64(9223372036854775807)), ([(NULL, 100), (NULL, NULL), (1024, 10)], toUInt64(2), toUInt64(1024)), ''), hex(sipHash128ReferenceKeyed((-9223372036854775807, 1.), '-1', NULL)), ('', toUInt64(65535), [(9223372036854775807, 9223372036854775806)], toUInt64(65536)), arrayJoin((NULL, 65537, 255), [(NULL, NULL)]) GROUP BY tupleElement((NULL, NULL, NULL, -1), toUInt64(2), 2) = NULL; -- { serverError NOT_IMPLEMENTED } SELECT hex(sipHash128ReferenceKeyed((0::UInt64, 0::UInt64), ([1, 1]))); -SELECT 'Check bug 2 found fuzzing'; +SELECT 'Test arrays and maps'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (`a` Map(String, String)) ENGINE = Memory; INSERT INTO sipHashKeyed_keys FORMAT VALUES ({'a':'b', 'c':'d'}), ({'e':'f', 'g':'h'}); SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FROM sipHashKeyed_keys ORDER BY a; DROP TABLE sipHashKeyed_keys; -SELECT 'Check bug 3 found fuzzing'; +SELECT 'Test emtpy arrays and maps'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []::Array(UInt8)); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), []::Array(UInt8))); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); From 027f913a139b39023633dbe038e4ee58d3141950 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 14 Aug 2024 11:17:04 +0200 Subject: [PATCH 201/409] Do not check i before it gets its final value --- src/Functions/FunctionsHashing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index c01ad5cf7e0..8d1c41f4c5f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -92,7 +92,6 @@ namespace impl if (is_const) i = 0; assert(key0->size() == key1->size()); - assert(key0->size() > i); if (offsets != nullptr) { const auto *const begin = offsets->begin(); @@ -102,6 +101,7 @@ namespace impl } const auto & key0data = assert_cast(*key0).getData(); const auto & key1data = assert_cast(*key1).getData(); + assert(key0->size() > i); return {key0data[i], key1data[i]}; } }; From 0b68517279dec4cdd1468719c846d17ef85629d2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 20 Aug 2024 20:01:35 +0200 Subject: [PATCH 202/409] skip projections --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeList.cpp | 19 ++++++++++++++----- src/Storages/MergeTree/MergeList.h | 2 ++ src/Storages/MergeTree/MergeTask.cpp | 2 +- 4 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..a48a04b4476 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -807,7 +807,7 @@ MergeTreeDataPartBuilder IMergeTreeDataPart::getProjectionPartBuilder(const Stri const char * projection_extension = is_temp_projection ? ".tmp_proj" : ".proj"; auto projection_storage = getDataPartStorage().getProjection(projection_name + projection_extension, !is_temp_projection); MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage); - return builder.withPartInfo({"all", 0, 0, 0}).withParentPart(this); + return builder.withPartInfo(MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION).withParentPart(this); } void IMergeTreeDataPart::addProjectionPart( diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 2465222ae6c..ed58b29d584 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -15,6 +15,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +const MergeTreePartInfo MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION = {"all", 0, 0, 0}; + MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context) : table_id{table_id_} , partition_id{future_part->part_info.partition_id} @@ -30,12 +32,18 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (result_part_name != result_part_info.getPartNameV1()) format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; + /// FIXME why do we need a merge list element for projection parts at all? + bool skip_sanity_checks = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION; + size_t normal_parts_count = 0; for (const auto & source_part : future_part->parts) { - normal_parts_count += !source_part->getParentPart(); - if (!source_part->getParentPart() && !result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); + if (!skip_sanity_checks && !source_part->getParentPart()) + { + ++normal_parts_count; + if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); + } source_part_names.emplace_back(source_part->name); source_part_paths.emplace_back(source_part->getDataPartStorage().getFullPath()); @@ -56,8 +64,9 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta part->partition.serializeText(part->storage, out, {}); } - if (is_mutation && normal_parts_count != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}", future_part->parts.size(), result_part_info.getPartNameV1()); + if (!skip_sanity_checks && is_mutation && normal_parts_count != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}: {}", future_part->parts.size(), + result_part_info.getPartNameV1(), fmt::join(source_part_names, ", ")); thread_group = ThreadGroup::createForBackgroundProcess(context); } diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index d40af6abf43..66190de0ef4 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -66,6 +66,8 @@ struct Settings; struct MergeListElement : boost::noncopyable { + static const MergeTreePartInfo FAKE_RESULT_PART_FOR_PROJECTION; + const StorageID table_id; std::string partition_id; std::string partition; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 26cb821f33b..f7cb01653c5 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -889,7 +889,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c // 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/"; - projection_future_part->part_info = {"all", 0, 0, 0}; + projection_future_part->part_info = MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION; MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; From 434458cc830d2ced68f1f96dcfa13f967c9bc74e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Aug 2024 11:22:22 +0000 Subject: [PATCH 203/409] Remove -n / --multiquery --- .../00115_shard_in_incomplete_result.sh | 2 +- .../0_stateless/00550_join_insert_select.sh | 2 +- .../0_stateless/01086_window_view_cleanup.sh | 2 +- ...396_inactive_replica_cleanup_nodes_zookeeper.sh | 4 ++-- .../0_stateless/01455_opentelemetry_distributed.sh | 2 +- .../0_stateless/01509_format_raw_blob.reference | 4 ++-- tests/queries/0_stateless/01509_format_raw_blob.sh | 4 ++-- .../01565_query_loop_after_client_error.expect | 2 +- .../01811_storage_buffer_flush_parameters.sh | 6 +++--- ...3_correct_block_size_prediction_with_default.sh | 4 ++-- .../02020_alter_table_modify_comment.sh | 2 +- ...e_sorting_by_input_stream_properties_explain.sh | 4 ++-- .../0_stateless/02383_join_and_filtering_set.sh | 2 +- .../0_stateless/02423_ddl_for_opentelemetry.sh | 2 +- tests/queries/0_stateless/02539_settings_alias.sh | 4 ++-- .../02697_stop_reading_on_first_cancel.sh | 2 +- .../0_stateless/02703_row_policies_for_asterisk.sh | 2 +- .../02703_row_policies_for_database_combination.sh | 2 +- .../0_stateless/02703_row_policy_for_database.sh | 4 ++-- tests/queries/0_stateless/02724_delay_mutations.sh | 6 +++--- ...02765_queries_with_subqueries_profile_events.sh | 14 +++++++------- .../queries/0_stateless/02841_not_ready_set_bug.sh | 4 ++-- .../0_stateless/02871_peak_threads_usage.sh | 14 +++++++------- .../0_stateless/02911_backup_restore_keeper_map.sh | 6 +++--- .../0_stateless/02968_file_log_multiple_read.sh | 4 ++-- .../03002_part_log_rmt_fetch_merge_error.sh | 8 ++++---- .../03002_part_log_rmt_fetch_mutate_error.sh | 10 +++++----- .../03164_selects_with_pk_usage_profile_event.sh | 8 ++++---- .../0_stateless/03172_system_detached_tables.sh | 4 ++-- .../03173_parallel_replicas_join_bug.sh | 2 +- 30 files changed, 68 insertions(+), 68 deletions(-) diff --git a/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh b/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh index 5c3918dea9f..4916721764c 100755 --- a/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh +++ b/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -o errexit set -o pipefail -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS users; CREATE TABLE users (UserID UInt64) ENGINE = Log; INSERT INTO users VALUES (1468013291393583084); diff --git a/tests/queries/0_stateless/00550_join_insert_select.sh b/tests/queries/0_stateless/00550_join_insert_select.sh index bfaccb613ca..ee2f3ab286b 100755 --- a/tests/queries/0_stateless/00550_join_insert_select.sh +++ b/tests/queries/0_stateless/00550_join_insert_select.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --ignore-error --query=" +$CLICKHOUSE_CLIENT --ignore-error --query=" DROP TABLE IF EXISTS test1_00550; DROP TABLE IF EXISTS test2_00550; DROP TABLE IF EXISTS test3_00550; diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 8b8e794c8ff..1bfa3c50869 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -13,7 +13,7 @@ opts=( DATABASE_ORDINARY="${CLICKHOUSE_DATABASE}_ordinary" -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 --multiquery " +$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " SET allow_experimental_window_view = 1; SET window_view_clean_interval = 1; diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index bff85b3e29f..4a0b6a8c93c 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -13,7 +13,7 @@ REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") SCALE=1000 -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x @@ -46,7 +46,7 @@ $CLICKHOUSE_CLIENT --receive_timeout 600 --query "SYSTEM SYNC REPLICA r2" # Need $CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/2$REPLICA' AND name = 'is_lost'"; -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; " diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh index 2b6da6132ed..30940f93a56 100755 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function check_log { -${CLICKHOUSE_CLIENT} --format=JSONEachRow -nq " +${CLICKHOUSE_CLIENT} --format=JSONEachRow -q " set enable_analyzer = 1; system flush logs; diff --git a/tests/queries/0_stateless/01509_format_raw_blob.reference b/tests/queries/0_stateless/01509_format_raw_blob.reference index 05014001bd9..eb074457e07 100644 --- a/tests/queries/0_stateless/01509_format_raw_blob.reference +++ b/tests/queries/0_stateless/01509_format_raw_blob.reference @@ -1,2 +1,2 @@ -9fd46251e5574c633cbfbb9293671888 - -9fd46251e5574c633cbfbb9293671888 - +48fad37bc89fc3bcc29c4750897b6709 - +48fad37bc89fc3bcc29c4750897b6709 - diff --git a/tests/queries/0_stateless/01509_format_raw_blob.sh b/tests/queries/0_stateless/01509_format_raw_blob.sh index 3d1d3fbb17b..355928014e8 100755 --- a/tests/queries/0_stateless/01509_format_raw_blob.sh +++ b/tests/queries/0_stateless/01509_format_raw_blob.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} cat ${BASH_SOURCE[0]} | md5sum -${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum +${CLICKHOUSE_CLIENT} --query "SELECT * FROM t FORMAT RawBLOB" | md5sum ${CLICKHOUSE_CLIENT} --query " DROP TABLE t; diff --git a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect index 6253840c63c..f08ef911da4 100755 --- a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect +++ b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect @@ -24,7 +24,7 @@ expect_after { -i $any_spawn_id timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=$history_file --highlight 0" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -m --history_file=$history_file --highlight 0" expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\r" diff --git a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh index 6a5949741ab..7878867e159 100755 --- a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh +++ b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh @@ -17,7 +17,7 @@ function wait_with_limit() done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_01811; drop table if exists buffer_01811; @@ -39,9 +39,9 @@ $CLICKHOUSE_CLIENT -nm -q " # wait for background buffer flush wait_with_limit 30 '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01811") -gt 0 ]]' -$CLICKHOUSE_CLIENT -nm -q "select count() from data_01811" +$CLICKHOUSE_CLIENT -m -q "select count() from data_01811" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table buffer_01811; drop table data_01811; " diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 1482730af2c..57f9b5595de 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) sql="toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]))" # Create the table and fill it -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=(\"[^\"]*?\"|[^\",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); ALTER TABLE test_extract ADD COLUMN 15Id Nullable(UInt16) DEFAULT $sql;" @@ -24,7 +24,7 @@ function test() $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 uuid_2=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" SYSTEM FLUSH LOGS; WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) diff --git a/tests/queries/0_stateless/02020_alter_table_modify_comment.sh b/tests/queries/0_stateless/02020_alter_table_modify_comment.sh index 3448f052f51..fa2d84e131a 100755 --- a/tests/queries/0_stateless/02020_alter_table_modify_comment.sh +++ b/tests/queries/0_stateless/02020_alter_table_modify_comment.sh @@ -16,7 +16,7 @@ function test_table_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT -nm <&1 | grep -q "SYNTAX_ERROR" -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON * USING x=1 AS permissive TO ALL; diff --git a/tests/queries/0_stateless/02724_delay_mutations.sh b/tests/queries/0_stateless/02724_delay_mutations.sh index f349e29253a..7843e692822 100755 --- a/tests/queries/0_stateless/02724_delay_mutations.sh +++ b/tests/queries/0_stateless/02724_delay_mutations.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_delay_mutations SYNC; CREATE TABLE t_delay_mutations (id UInt64, v UInt64) @@ -36,14 +36,14 @@ SELECT count() FROM system.mutations WHERE database = currentDatabase() AND tabl ${CLICKHOUSE_CLIENT} --query "SYSTEM START MERGES t_delay_mutations" wait_for_mutation "t_delay_mutations" "mutation_5.txt" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_delay_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_delay_mutations' AND NOT is_done; DROP TABLE IF EXISTS t_delay_mutations SYNC; " -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SYSTEM FLUSH LOGS; SELECT diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh index b7d93b5396c..fd64e8d8cb8 100755 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS mv; DROP TABLE IF EXISTS output; DROP TABLE IF EXISTS input; @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -n -q " for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "INSERT INTO input SELECT * FROM numbers(1)" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 view, @@ -35,7 +35,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "SELECT * FROM system.one WHERE dummy IN (SELECT * FROM system.one) FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 subquery, @@ -52,7 +52,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CSE, @@ -69,7 +69,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x, x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CSE_Multi, @@ -86,7 +86,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CTE, @@ -103,7 +103,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x UNION ALL SELECT * FROM x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CTE_Multi, diff --git a/tests/queries/0_stateless/02841_not_ready_set_bug.sh b/tests/queries/0_stateless/02841_not_ready_set_bug.sh index 556e2f52de2..d5a2d034014 100755 --- a/tests/queries/0_stateless/02841_not_ready_set_bug.sh +++ b/tests/queries/0_stateless/02841_not_ready_set_bug.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE 1 in (SELECT number fro $CLICKHOUSE_CLIENT -q "SELECT xor(1, 0) FROM system.parts WHERE 1 IN (SELECT 1) FORMAT Null" # (Not all of these tests are effective because some of these tables are empty.) -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " select * from system.columns where table in (select '123'); select * from system.replicas where database in (select '123'); select * from system.data_skipping_indices where database in (select '123'); @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -nq " select * from system.replication_queue where database in (select '123'); select * from system.distribution_queue where database in (select '123'); " -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " create table a (x Int8) engine MergeTree order by x; insert into a values (1); select * from mergeTreeIndex(currentDatabase(), 'a') where part_name in (select '123'); diff --git a/tests/queries/0_stateless/02871_peak_threads_usage.sh b/tests/queries/0_stateless/02871_peak_threads_usage.sh index dfb3e665020..0f0473bbb47 100755 --- a/tests/queries/0_stateless/02871_peak_threads_usage.sh +++ b/tests/queries/0_stateless/02871_peak_threads_usage.sh @@ -26,7 +26,7 @@ ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_6" --query='SELECT * FROM nu ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_7" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 1, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_8" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 4, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_9" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_9" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(10000, 200000) @@ -38,7 +38,7 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(300000, 4000000) ) SETTINGS max_threads = 1""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_10" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_10" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(10000, 2000) @@ -50,7 +50,7 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(300000, 4000000) ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_11" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_11" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(1, 1) @@ -62,20 +62,20 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(1, 4000000) ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_12" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_12" -m --query=""" SELECT sum(number) FROM numbers_mt(100000) GROUP BY number % 2 WITH TOTALS ORDER BY number % 2 SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_13" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 1" "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_13" -m --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 1" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_14" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 4" "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_14" -m --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 4" "${QUERY_OPTIONS[@]}" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" for i in {1..14} do - ${CLICKHOUSE_CLIENT} -mn --query=""" + ${CLICKHOUSE_CLIENT} -m --query=""" SELECT '${i}', peak_threads_usage, (select count() from system.query_thread_log WHERE system.query_thread_log.query_id = '${UNIQUE_QUERY_ID}_${i}' AND current_database = currentDatabase()) = length(thread_ids), diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index c04667505c3..01aba244a02 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh database_name="$CLICKHOUSE_DATABASE"_02911_keeper_map -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP DATABASE IF EXISTS $database_name; CREATE DATABASE $database_name; CREATE TABLE $database_name.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; @@ -13,9 +13,9 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; " -$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" +$CLICKHOUSE_CLIENT -m -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" -$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" +$CLICKHOUSE_CLIENT -m -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" backup_path="$database_name" for i in $(seq 1 3); do diff --git a/tests/queries/0_stateless/02968_file_log_multiple_read.sh b/tests/queries/0_stateless/02968_file_log_multiple_read.sh index d9bae05270a..0879bf02d60 100755 --- a/tests/queries/0_stateless/02968_file_log_multiple_read.sh +++ b/tests/queries/0_stateless/02968_file_log_multiple_read.sh @@ -15,7 +15,7 @@ do echo $i >> ${logs_dir}/a.txt done -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS file_log; DROP TABLE IF EXISTS table_to_store_data; DROP TABLE IF EXISTS file_log_mv; @@ -69,7 +69,7 @@ done ${CLICKHOUSE_CLIENT} --query "SELECT * FROM table_to_store_data ORDER BY id;" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE file_log; DROP TABLE table_to_store_data; DROP TABLE file_log_mv; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh index e58c542b8ac..185e46a2eac 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -11,13 +11,13 @@ set -e function wait_until() { local q=$1 && shift - while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do # too frequent FLUSH LOGS is too costly sleep 2 done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists rmt_master; drop table if exists rmt_slave; @@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT -nm -q " optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; @@ -42,7 +42,7 @@ $CLICKHOUSE_CLIENT -nm -q " " # wait until rmt_slave will fetch the part and reflect this error in system.part_log wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system sync replica rmt_slave; system flush logs; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh index cc8f53aafb9..e731d51e7e3 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh @@ -14,13 +14,13 @@ set -e function wait_until() { local q=$1 && shift - while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do # too frequent FLUSH LOGS is too costly sleep 2 done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists rmt_master; drop table if exists rmt_slave; @@ -41,10 +41,10 @@ $CLICKHOUSE_CLIENT -nm -q " # the part, and rmt_slave will consider it instead of performing mutation on # it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored wait_for_mutation rmt_master 0000000000 -$CLICKHOUSE_CLIENT -nm -q "system start pulling replication log rmt_slave" +$CLICKHOUSE_CLIENT -m -q "system start pulling replication log rmt_slave" # and wait until rmt_slave to fetch the part and reflect this error in system.part_log wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nm -q " system start replicated sends rmt_master; " wait_for_mutation rmt_slave 0000000000 -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system sync replica rmt_slave; system flush logs; diff --git a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh index 29d4c877909..75efc3f057a 100755 --- a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh +++ b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh @@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -50,7 +50,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE col2 >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -67,7 +67,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE pk >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -84,7 +84,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE col1 >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sh b/tests/queries/0_stateless/03172_system_detached_tables.sh index 47775abcc45..60e913b62a8 100755 --- a/tests/queries/0_stateless/03172_system_detached_tables.sh +++ b/tests/queries/0_stateless/03172_system_detached_tables.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATABASE_ATOMIC="${CLICKHOUSE_DATABASE}_atomic" DATABASE_LAZY="${CLICKHOUSE_DATABASE}_lazy" -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " SELECT 'database atomic tests'; DROP DATABASE IF EXISTS ${DATABASE_ATOMIC}; @@ -36,7 +36,7 @@ DROP DATABASE ${DATABASE_ATOMIC} SYNC; " -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " SELECT '-----------------------'; SELECT 'database lazy tests'; diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index af702569794..d2be9899f86 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -q " INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); " -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " SET enable_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; SELECT From 4c6f30a70df20bb53625aceb1eb5256664f99080 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 20 Aug 2024 17:46:29 +0000 Subject: [PATCH 204/409] Cosmetics --- src/Functions/FunctionsHashing.h | 50 +++++++++++-------- .../0_stateless/02534_keyed_siphash.sql | 12 ++--- .../02552_siphash128_reference.sql | 4 +- 3 files changed, 36 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 8d1c41f4c5f..0cf4246fd66 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -77,7 +77,7 @@ namespace impl ColumnPtr key0; ColumnPtr key1; bool is_const; - const ColumnArray::Offsets * offsets{}; + const ColumnArray::Offsets * offsets = nullptr; size_t size() const { @@ -87,6 +87,7 @@ namespace impl return offsets->back(); return key0->size(); } + SipHashKey getKey(size_t i) const { if (is_const) @@ -94,7 +95,7 @@ namespace impl assert(key0->size() == key1->size()); if (offsets != nullptr) { - const auto *const begin = offsets->begin(); + const auto * const begin = offsets->begin(); const auto * upper = std::upper_bound(begin, offsets->end(), i); if (upper != offsets->end()) i = upper - begin; @@ -108,33 +109,38 @@ namespace impl static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key) { - const ColumnTuple * tuple = nullptr; - const auto * column = key.column.get(); - bool is_const = false; - if (isColumnConst(*column)) + const auto * col_key = key.column.get(); + + bool is_const; + const ColumnTuple * col_key_tuple; + if (isColumnConst(*col_key)) { is_const = true; - tuple = checkAndGetColumnConstData(column); + col_key_tuple = checkAndGetColumnConstData(col_key); } else - tuple = checkAndGetColumn(column); - if (!tuple) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple"); - if (tuple->tupleSize() != 2) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); + { + is_const = false; + col_key_tuple = checkAndGetColumn(col_key); + } - SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const}; - assert(ret.key0); - if (!checkColumn(*ret.key0)) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); - assert(ret.key1); - if (!checkColumn(*ret.key1)) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); + if (!col_key_tuple || col_key_tuple->tupleSize() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The key must be of type Tuple(UInt64, UInt64)"); - if (ret.size() == 1) - ret.is_const = true; + SipHashKeyColumns result{.key0 = col_key_tuple->getColumnPtr(0), .key1 = col_key_tuple->getColumnPtr(1), .is_const = is_const}; - return ret; + assert(result.key0); + assert(result.key1); + + if (!checkColumn(*result.key0)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The 1st element of the key tuple is not of type UInt64"); + if (!checkColumn(*result.key1)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The 2nd element of the key tuple is not of type UInt64"); + + if (result.size() == 1) + result.is_const = true; + + return result; } } diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index b96233200a8..b499d8ef02b 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash64Keyed((0, 0), '1'); -- { serverError BAD_ARGUMENTS } +select sipHash128Keyed((0, 0), '1'); -- { serverError BAD_ARGUMENTS } +select sipHash64Keyed(toUInt64(0), '1'); -- { serverError BAD_ARGUMENTS } +select sipHash128Keyed(toUInt64(0), '1'); -- { serverError BAD_ARGUMENTS } select hex(sipHash64()); SELECT hex(sipHash128()); @@ -347,8 +347,8 @@ SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FRO DROP TABLE sipHashKeyed_keys; SELECT 'Test emtpy arrays and maps'; -SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []::Array(UInt8)); -SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), []::Array(UInt8))); +SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3)); diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index f7324ed0ee4..46f292d667d 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -200,8 +200,8 @@ select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError BAD_ARGUMENTS } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError BAD_ARGUMENTS } SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From 4b08ae5f650313d286d416c68d92c240677ceb0f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Aug 2024 21:51:48 +0200 Subject: [PATCH 205/409] Restart CI --- tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 index 263e92be403..dd60a31f771 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 +++ b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 @@ -2,6 +2,7 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set allow_experimental_dynamic_type = 1; + drop table if exists test; {% for engine in ['MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', From 9fd9f649db6bd3149cda217a200cfee3d7bc8238 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Aug 2024 21:53:42 +0200 Subject: [PATCH 206/409] Add comment in IDataType.cpp --- src/DataTypes/IDataType.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index d14461a4ff1..1a274c7f993 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -150,6 +150,7 @@ std::unique_ptr IDataType::getSubcolumnData( ISerialization::EnumerateStreamsSettings settings; settings.position_independent_encoding = false; + /// Don't enumerate dynamic subcolumns, they are handled separately. settings.enumerate_dynamic_streams = false; data.serialization->enumerateStreams(settings, callback_with_data, data); From 5d280053c2512332bc0c171e57dbc36cb3e0b675 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Aug 2024 21:55:10 +0200 Subject: [PATCH 207/409] Add comment in ISerialization.h --- src/DataTypes/Serializations/ISerialization.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index d19cb0dd365..33575a07177 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -241,6 +241,9 @@ public: { SubstreamPath path; bool position_independent_encoding = true; + /// If set to false, don't enumerate dynamic subcolumns + /// (such as dynamic types in Dynamic column or dynamic paths in JSON column). + /// It may be needed when dynamic subcolumns are processed separately. bool enumerate_dynamic_streams = true; }; From 539d04c90f30efa0ef6435373ec8ffc4777aee78 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Aug 2024 20:00:23 +0000 Subject: [PATCH 208/409] 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 209/409] 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 210/409] 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 7fe98ac30224d033110faa90b312f4badc25ca32 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 20 Aug 2024 22:20:26 +0000 Subject: [PATCH 211/409] Fix stress_tests.lib potentially mistaking its own search string for a crash message --- tests/docker_scripts/stress_tests.lib | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index e2b5d983fcb..4f3e6eeb2f4 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -273,7 +273,7 @@ function check_logs_for_critical_errors() [ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt # Crash - rg -Fa "########################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ + rg -Fa "###################""#####################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ && echo -e "Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \ || echo -e "Not crashed$OK" >> /test_output/test_results.tsv @@ -285,7 +285,7 @@ function check_logs_for_critical_errors() # Remove file fatal_messages.txt if it's empty [ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt - rg -Faz "########################################" /test_output/* > /dev/null \ + rg -Faz "####################""####################" /test_output/* > /dev/null \ && echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv function get_gdb_log_context() From bb2b6600961e432d115c08964d65ade23740861a Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 02:11:08 +0300 Subject: [PATCH 212/409] 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 213/409] 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 3c7e2389d1207582a81ac28f66bd28ed9329c489 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 09:14:50 +0800 Subject: [PATCH 214/409] Fix code style --- .../test_incorrect_datetime_format/test.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py index c1803aa95a1..56c061f3830 100644 --- a/tests/integration/test_incorrect_datetime_format/test.py +++ b/tests/integration/test_incorrect_datetime_format/test.py @@ -2,6 +2,7 @@ import logging import pytest from helpers.cluster import ClickHouseCluster + @pytest.fixture(scope="module") def cluster(): try: @@ -10,7 +11,7 @@ def cluster(): "node", main_configs=[ "configs/config.d/cluster.xml", - ] + ], ) logging.info("Starting cluster...") cluster.start() @@ -28,7 +29,8 @@ def test_incorrect_datetime_format(cluster): node = cluster.instances["node"] - node.query(""" + node.query( + """ CREATE TABLE tab ( a DateTime, @@ -40,10 +42,12 @@ def test_incorrect_datetime_format(cluster): res = node.query("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'").strip() assert res == "0" - error = node.query_and_get_error("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'").strip() - print(error) + error = node.query_and_get_error( + "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'" + ).strip() assert "Cannot parse time component of DateTime 09:58:0" in error - error = node.query_and_get_error("SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'").strip() - print(error) + error = node.query_and_get_error( + "SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'" + ).strip() assert "Cannot convert string '2024-08-0 09:58:09' to type DateTime" in error From 471320dc6f8781e43e5d507086a72a6552caf531 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 14:15:44 +0800 Subject: [PATCH 215/409] Fix flaky test error --- .../test_incorrect_datetime_format/test.py | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py index 56c061f3830..3cdc6781534 100644 --- a/tests/integration/test_incorrect_datetime_format/test.py +++ b/tests/integration/test_incorrect_datetime_format/test.py @@ -17,6 +17,17 @@ def cluster(): cluster.start() logging.info("Cluster started") + node = cluster.instances["node"] + node.query( + """ + CREATE TABLE tab + ( + a DateTime, + pk String + ) Engine = MergeTree() ORDER BY pk; + """ + ) + yield cluster finally: cluster.shutdown() @@ -29,16 +40,6 @@ def test_incorrect_datetime_format(cluster): node = cluster.instances["node"] - node.query( - """ - CREATE TABLE tab - ( - a DateTime, - pk String - ) Engine = MergeTree() ORDER BY pk; - """ - ) - res = node.query("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'").strip() assert res == "0" From e01a448bcc62a7e292766cddc0c817b9e44558d4 Mon Sep 17 00:00:00 2001 From: Zhigao Hong Date: Wed, 21 Aug 2024 15:35:33 +0800 Subject: [PATCH 216/409] Fix invalid characters in replica_name --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 44548e33d46..9a65d590453 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -538,6 +538,9 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replica_name.empty()) throw Exception(ErrorCodes::NO_REPLICA_NAME_GIVEN, "No replica name in config{}", verbose_help_message); + // '\t' and '\n' will interrupt parsing 'source replica' in ReplicatedMergeTreeLogEntryData::readText + if (replica_name.find('\t') != String::npos || replica_name.find('\n') != String::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must not contain '\\t' or '\\n'"); arg_cnt = engine_args.size(); /// Update `arg_cnt` here because extractZooKeeperPathAndReplicaNameFromEngineArgs() could add arguments. arg_num = 2; /// zookeeper_path and replica_name together are always two arguments. From f12840eb5f289aca0d8ada4cde4c902be00de993 Mon Sep 17 00:00:00 2001 From: siyuan Date: Wed, 21 Aug 2024 15:45:58 +0800 Subject: [PATCH 217/409] Fix wrong row id when using full text index with multi column --- .../MergeTree/MergeTreeIndexFullText.cpp | 4 ++-- .../03228_full_text_with_multi_col.reference | 2 ++ .../03228_full_text_with_multi_col.sql | 18 ++++++++++++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03228_full_text_with_multi_col.reference create mode 100644 tests/queries/0_stateless/03228_full_text_with_multi_col.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index b5c6bb95d37..ba3ca2f139a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -128,14 +128,14 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); - auto row_id = store->getNextRowIDRange(rows_read); - auto start_row_id = row_id; + auto start_row_id = store->getNextRowIDRange(rows_read); for (size_t col = 0; col < index_columns.size(); ++col) { const auto & column_with_type = block.getByName(index_columns[col]); const auto & column = column_with_type.column; size_t current_position = *pos; + auto row_id = start_row_id; bool need_to_write = false; if (isArray(column_with_type.type)) diff --git a/tests/queries/0_stateless/03228_full_text_with_multi_col.reference b/tests/queries/0_stateless/03228_full_text_with_multi_col.reference new file mode 100644 index 00000000000..9f721a82e59 --- /dev/null +++ b/tests/queries/0_stateless/03228_full_text_with_multi_col.reference @@ -0,0 +1,2 @@ +Query column at granularity boundary +0,0 0,1 diff --git a/tests/queries/0_stateless/03228_full_text_with_multi_col.sql b/tests/queries/0_stateless/03228_full_text_with_multi_col.sql new file mode 100644 index 00000000000..e1c78c5ffda --- /dev/null +++ b/tests/queries/0_stateless/03228_full_text_with_multi_col.sql @@ -0,0 +1,18 @@ +SET allow_experimental_full_text_index=1; + +DROP TABLE IF EXISTS multi_col_ivt; + +CREATE TABLE tab ( + v0 String, + v1 String, + INDEX idx (v0, v1) TYPE full_text GRANULARITY 1) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 1; + +INSERT INTO tab VALUES('0,0', '0,1')('2,2','2,3'); + +SELECT 'Query column at granularity boundary'; +SELECT * FROM tab WHERE hasToken(v1, '1'); + +DROP TABLE tab; From 47a245cb65e6550769f6a3621a8902b43bfd9160 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 21 Aug 2024 12:11:47 +0200 Subject: [PATCH 218/409] Remove wrong release version --- docs/changelogs/v23.8.16.16-lts.md | 38 ---------------------------- utils/list-versions/version_date.tsv | 1 - 2 files changed, 39 deletions(-) delete mode 100644 docs/changelogs/v23.8.16.16-lts.md diff --git a/docs/changelogs/v23.8.16.16-lts.md b/docs/changelogs/v23.8.16.16-lts.md deleted file mode 100644 index 9532db4fb0a..00000000000 --- a/docs/changelogs/v23.8.16.16-lts.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -sidebar_position: 1 -sidebar_label: 2024 ---- - -# 2024 Changelog - -### ClickHouse release v23.8.16.16-lts (b80cac57ead) FIXME as compared to v23.8.15.35-lts (060ff8e813a) - -#### Improvement -* Backported in [#66962](https://github.com/ClickHouse/ClickHouse/issues/66962): Added support for parameterized view with analyzer to not analyze create parameterized view. Refactor existing parameterized view logic to not analyze create parameterized view. [#54211](https://github.com/ClickHouse/ClickHouse/pull/54211) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Backported in [#65461](https://github.com/ClickHouse/ClickHouse/issues/65461): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). -* Backported in [#65880](https://github.com/ClickHouse/ClickHouse/issues/65880): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). -* Backported in [#65912](https://github.com/ClickHouse/ClickHouse/issues/65912): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). - -#### Bug Fix (user-visible misbehavior in an official stable release) -* Backported in [#65351](https://github.com/ClickHouse/ClickHouse/issues/65351): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). -* Backported in [#66037](https://github.com/ClickHouse/ClickHouse/issues/66037): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). -* Backported in [#65281](https://github.com/ClickHouse/ClickHouse/issues/65281): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). -* Backported in [#65368](https://github.com/ClickHouse/ClickHouse/issues/65368): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). -* Backported in [#65782](https://github.com/ClickHouse/ClickHouse/issues/65782): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). -* Backported in [#65743](https://github.com/ClickHouse/ClickHouse/issues/65743): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). -* Backported in [#65926](https://github.com/ClickHouse/ClickHouse/issues/65926): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). -* Backported in [#65822](https://github.com/ClickHouse/ClickHouse/issues/65822): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). -* Backported in [#66322](https://github.com/ClickHouse/ClickHouse/issues/66322): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). -* Backported in [#66449](https://github.com/ClickHouse/ClickHouse/issues/66449): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Backported in [#66717](https://github.com/ClickHouse/ClickHouse/issues/66717): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). -* Backported in [#67320](https://github.com/ClickHouse/ClickHouse/issues/67320): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). - -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Backported in [#65080](https://github.com/ClickHouse/ClickHouse/issues/65080): Follow up to [#56541](https://github.com/ClickHouse/ClickHouse/issues/56541). [#57141](https://github.com/ClickHouse/ClickHouse/pull/57141) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Backported in [#64997](https://github.com/ClickHouse/ClickHouse/issues/64997): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). -* Backported in [#65913](https://github.com/ClickHouse/ClickHouse/issues/65913): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). -* Backported in [#66853](https://github.com/ClickHouse/ClickHouse/issues/66853): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). -* Backported in [#67072](https://github.com/ClickHouse/ClickHouse/issues/67072): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). -* Update version after release. [#67691](https://github.com/ClickHouse/ClickHouse/pull/67691) ([robot-clickhouse](https://github.com/robot-clickhouse)). - diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 776a53ec01c..95ef8c0de90 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -56,7 +56,6 @@ v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 v23.8.16.40-lts 2024-08-02 -v23.8.16.16-lts 2024-08-20 v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 From 915daafd3a0c9f1539dad75dc3805e740f0bc75a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 10:45:48 +0000 Subject: [PATCH 219/409] Fix 01086_window_view_cleanup.sh --- tests/queries/0_stateless/01086_window_view_cleanup.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 1bfa3c50869..2e6cc7e2520 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -13,7 +13,8 @@ opts=( DATABASE_ORDINARY="${CLICKHOUSE_DATABASE}_ordinary" -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " +$CLICKHOUSE_CLIENT "${opts[@]}" --query " + SET allow_deprecated_database_ordinary = 1; SET allow_experimental_window_view = 1; SET window_view_clean_interval = 1; @@ -28,8 +29,7 @@ $CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 2, toDateTime('1990/01/01 12:00:01', 'US/Samoa')); INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 3, toDateTime('1990/01/01 12:00:02', 'US/Samoa')); INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 4, toDateTime('1990/01/01 12:00:05', 'US/Samoa')); - INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 5, toDateTime('1990/01/01 12:00:06', 'US/Samoa')); -" + INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 5, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" while true; do $CLICKHOUSE_CLIENT "${opts[@]}" --query="SELECT count(*) FROM ${DATABASE_ORDINARY}.\`.inner.wv\`" | grep -q "5" && break || sleep .5 ||: From 8bf103e82f62b1484801fd95f7f73a42a33f7fef Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 21 Aug 2024 12:49:33 +0200 Subject: [PATCH 220/409] squash! materialized_view_deduplication performance comparison test performance comparison test to check deduplication in MATERIALIZED VIEW's. Logic is similar to, but with a bigger insert tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql --- tests/performance/materialized_view_deduplication.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/materialized_view_deduplication.xml b/tests/performance/materialized_view_deduplication.xml index 621effd23dd..e5e0e5fc6e4 100644 --- a/tests/performance/materialized_view_deduplication.xml +++ b/tests/performance/materialized_view_deduplication.xml @@ -2,7 +2,6 @@ 1 - CREATE TABLE dst (`key` Int64, `value` String) ENGINE = MergeTree ORDER BY tuple() From c8bc7a124046ed4789ffdd6ce953f7764f88df09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Aug 2024 14:52:11 +0200 Subject: [PATCH 221/409] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index bd9b8f66ad0..17b6dcd2ac1 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Guangzho User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 +* [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" 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 222/409] 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 223/409] 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 4a7a04b35b492aec779b42adc6f3f3eae354a947 Mon Sep 17 00:00:00 2001 From: leonkozlowski Date: Wed, 21 Aug 2024 10:13:02 -0400 Subject: [PATCH 224/409] patch: build 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 225/409] 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 1afd3a7c3a7569b172ac3238f798c7850fd41bcf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 21 Aug 2024 16:24:43 +0200 Subject: [PATCH 226/409] give priority to parsed columns over storage columns --- src/Storages/Hive/StorageHive.cpp | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../StorageObjectStorageCluster.cpp | 2 +- .../StorageObjectStorageSource.cpp | 4 ++-- .../StorageObjectStorageSource.h | 4 ++-- .../ObjectStorageQueueSource.cpp | 2 +- .../ObjectStorageQueueSource.h | 2 +- .../StorageObjectStorageQueue.cpp | 2 +- src/Storages/StorageFile.cpp | 6 ++--- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageURL.cpp | 6 ++--- src/Storages/StorageURLCluster.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 23 +++++++++++-------- src/Storages/VirtualColumnUtils.h | 4 ++-- 14 files changed, 33 insertions(+), 30 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index ae2e8cffe28..ea2e9e3eece 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -444,8 +444,8 @@ StorageHive::StorageHive( storage_metadata.setComment(comment_); storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, getContext())); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), getContext())); } void StorageHive::lazyInitialize() diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index d9c82d68791..a0f189e92fc 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -94,7 +94,7 @@ StorageObjectStorage::StorageObjectStorage( if (sample_path.empty() && context->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.columns, context, sample_path, format_settings)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index c214665f7e0..08a0739d929 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -68,7 +68,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.columns, context_, sample_path)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index d8e26977e75..04e319cd0b8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -208,7 +208,7 @@ Chunk StorageObjectStorageSource::generate() .filename = &filename, .last_modified = object_info->metadata->last_modified, .etag = &(object_info->metadata->etag) - }, getContext(), read_from_format_info.columns_description); + }, getContext()); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) @@ -280,7 +280,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade const std::shared_ptr & file_iterator, const ConfigurationPtr & configuration, const ObjectStoragePtr & object_storage, - const ReadFromFormatInfo & read_from_format_info, + ReadFromFormatInfo & read_from_format_info, const std::optional & format_settings, const std::shared_ptr & key_condition_, const ContextPtr & context_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 6681dbf4578..7ae7a2358e9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -74,7 +74,7 @@ protected: const UInt64 max_block_size; const bool need_only_count; const size_t max_parsing_threads; - const ReadFromFormatInfo read_from_format_info; + ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; std::shared_ptr file_iterator; @@ -122,7 +122,7 @@ protected: const std::shared_ptr & file_iterator, const ConfigurationPtr & configuration, const ObjectStoragePtr & object_storage, - const ReadFromFormatInfo & read_from_format_info, + ReadFromFormatInfo & read_from_format_info, const std::optional & format_settings, const std::shared_ptr & key_condition_, const ContextPtr & context_, diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 2634a7b2f1e..cde41b4afff 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -524,7 +524,7 @@ Chunk ObjectStorageQueueSource::generateImpl() { .path = path, .size = reader.getObjectInfo()->metadata->size_bytes - }, getContext(), read_from_format_info.columns_description); + }, getContext()); return chunk; } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h index 0f3d0ab2e92..c085287e4f3 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h @@ -128,7 +128,7 @@ private: const std::shared_ptr file_iterator; const ConfigurationPtr configuration; const ObjectStoragePtr object_storage; - const ReadFromFormatInfo read_from_format_info; + ReadFromFormatInfo read_from_format_info; const std::optional format_settings; const ObjectStorageQueueSettings queue_settings; const std::shared_ptr files_metadata; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 5dc3e01962c..9452ce81e9e 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -169,7 +169,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context_)); setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 766b7722cdf..50294df32a4 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1112,9 +1112,9 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setConstraints(args.constraints); storage_metadata.setComment(args.comment); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), paths.empty() ? "" : paths[0], format_settings)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, args.getContext(), paths.empty() ? "" : paths[0], format_settings)); + setInMemoryMetadata(storage_metadata); } @@ -1468,7 +1468,7 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified - }, getContext(), columns_description); + }, getContext()); return chunk; } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index 82ae0b761ae..c01738067c4 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -60,8 +60,8 @@ StorageFileCluster::StorageFileCluster( } storage_metadata.setConstraints(constraints_); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context, paths.empty() ? "" : paths[0])); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, paths.empty() ? "" : paths[0])); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6442891cf23..fc1354b780a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -165,9 +165,9 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context_, getSampleURI(uri, context_), format_settings)); + setInMemoryMetadata(storage_metadata); } @@ -435,7 +435,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - }, getContext(), columns_description); + }, getContext()); return chunk; } diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 7c7a299c64e..140413d78b0 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,8 +75,8 @@ StorageURLCluster::StorageURLCluster( } storage_metadata.setConstraints(constraints_); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context, getSampleURI(uri, context))); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context))); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index edf50907752..5b974cb8a22 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -129,7 +129,7 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time", "_etag"}; } -std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) +std::unordered_map parseHivePartitioningKeysAndValues(const String & path) { std::string pattern = "([^/]+)=([^/]+)/"; re2::StringPiece input_piece(path); @@ -145,34 +145,37 @@ std::unordered_map parseHivePartitioningKeysAndValues( used_keys.insert({key, value}); auto col_name = key; - while (storage_columns.has(col_name)) - col_name = "_" + col_name; key_values[col_name] = value; } return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional format_settings_) +VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional format_settings_) { VirtualColumnsDescription desc; auto add_virtual = [&](const auto & name, const auto & type) { - if (storage_columns.has(name)) + auto local_type = type; + if (storage_columns.has(name) && !context->getSettingsRef().use_hive_partitioning) return; + if (storage_columns.has(name)) + { + local_type = storage_columns.get(name).type; + storage_columns.remove(name); + } - desc.addEphemeral(name, type, ""); + desc.addEphemeral(name, local_type, ""); }; add_virtual("_path", std::make_shared(std::make_shared())); add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - add_virtual("_etag", std::make_shared(std::make_shared())); if (context->getSettingsRef().use_hive_partitioning) { - auto map = parseHivePartitioningKeysAndValues(path, storage_columns); + auto map = parseHivePartitioningKeysAndValues(path); auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); for (auto & item : map) { @@ -245,11 +248,11 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns) + VirtualsForFileLikeStorage virtual_values, ContextPtr context) { std::unordered_map hive_map; if (context->getSettingsRef().use_hive_partitioning) - hive_map = parseHivePartitioningKeysAndValues(virtual_values.path, columns); + hive_map = parseHivePartitioningKeysAndValues(virtual_values.path); for (const auto & virtual_column : requested_virtual_columns) { diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 23e16871798..6aa08b2aef2 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -70,7 +70,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage( - const ColumnsDescription & storage_columns, + ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & sample_path = "", std::optional format_settings_ = std::nullopt); @@ -105,7 +105,7 @@ struct VirtualsForFileLikeStorage void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns); + VirtualsForFileLikeStorage virtual_values, ContextPtr context); } } From ccff8cef80394934e1567fd0c669b8f67fe7b660 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 21 Aug 2024 14:27:49 +0000 Subject: [PATCH 227/409] Update version_date.tsv and changelogs after v24.8.1.2684-lts --- SECURITY.md | 16 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.1.2684-lts.md | 525 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 532 insertions(+), 16 deletions(-) create mode 100644 docs/changelogs/v24.8.1.2684-lts.md diff --git a/SECURITY.md b/SECURITY.md index 8930dc96f8a..93c48f1d9ba 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,25 +14,15 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.8 | ✔️ | | 24.7 | ✔️ | | 24.6 | ✔️ | -| 24.5 | ✔️ | +| 24.5 | ❌ | | 24.4 | ❌ | | 24.3 | ✔️ | | 24.2 | ❌ | | 24.1 | ❌ | -| 23.12 | ❌ | -| 23.11 | ❌ | -| 23.10 | ❌ | -| 23.9 | ❌ | -| 23.8 | ✔️ | -| 23.7 | ❌ | -| 23.6 | ❌ | -| 23.5 | ❌ | -| 23.4 | ❌ | -| 23.3 | ❌ | -| 23.2 | ❌ | -| 23.1 | ❌ | +| 23.* | ❌ | | 22.* | ❌ | | 21.* | ❌ | | 20.* | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index a44664259fb..fc93cee5bbc 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.3.42" +ARG VERSION="24.8.1.2684" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 2565828c846..3ceaf2a08b4 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.3.42" +ARG VERSION="24.8.1.2684" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5ac8a58afea..76db997821c 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.7.3.42" +ARG VERSION="24.8.1.2684" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.1.2684-lts.md b/docs/changelogs/v24.8.1.2684-lts.md new file mode 100644 index 00000000000..8171bb3d719 --- /dev/null +++ b/docs/changelogs/v24.8.1.2684-lts.md @@ -0,0 +1,525 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.1.2684-lts (161c62fd295) FIXME as compared to v24.8.1.1-new (3f8b27d7acc) + +#### Backward Incompatible Change +* `clickhouse-client` and `clickhouse-local` now default to multi-query mode (instead single-query mode). As an example, `clickhouse-client -q "SELECT 1; SELECT 2"` now works, whereas users previously had to add `--multiquery` (or `-n`). The `--multiquery/-n` switch became obsolete. INSERT queries in multi-query statements are treated specially based on their FORMAT clause: If the FORMAT is `VALUES` (the most common case), the end of the INSERT statement is represented by a trailing semicolon `;` at the end of the query. For all other FORMATs (e.g. `CSV` or `JSONEachRow`), the end of the INSERT statement is represented by two newlines `\n\n` at the end of the query. [#63898](https://github.com/ClickHouse/ClickHouse/pull/63898) ([FFish](https://github.com/wxybear)). +* In previous versions, it was possible to use an alternative syntax for `LowCardinality` data types by appending `WithDictionary` to the name of the data type. It was an initial working implementation, and it was never documented or exposed to the public. Now, it is deprecated. If you have used this syntax, you have to ALTER your tables and rename the data types to `LowCardinality`. [#66842](https://github.com/ClickHouse/ClickHouse/pull/66842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix logical errors with storage `Buffer` used with distributed destination table. It's a backward incompatible change: queries using `Buffer` with a distributed destination table may stop working if the table appears more than once in the query (e.g., in a self-join). [#67015](https://github.com/ClickHouse/ClickHouse/pull/67015) ([vdimir](https://github.com/vdimir)). +* In previous versions, calling functions for random distributions based on the Gamma function (such as Chi-Squared, Student, Fisher) with negative arguments close to zero led to a long computation or an infinite loop. In the new version, calling these functions with zero or negative arguments will produce an exception. This closes [#67297](https://github.com/ClickHouse/ClickHouse/issues/67297). [#67326](https://github.com/ClickHouse/ClickHouse/pull/67326) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The system table `text_log` is enabled by default. This is fully compatible with previous versions, but you may notice subtly increased disk usage on the local disk (this system table takes a tiny amount of disk space). [#67428](https://github.com/ClickHouse/ClickHouse/pull/67428) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In previous versions, `arrayWithConstant` can be slow if asked to generate very large arrays. In the new version, it is limited to 1 GB per array. This closes [#32754](https://github.com/ClickHouse/ClickHouse/issues/32754). [#67741](https://github.com/ClickHouse/ClickHouse/pull/67741) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix REPLACE modifier formatting (forbid omitting brackets). [#67774](https://github.com/ClickHouse/ClickHouse/pull/67774) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68349](https://github.com/ClickHouse/ClickHouse/issues/68349): Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Kruglov Pavel](https://github.com/Avogar)). + +#### New Feature +* Add new experimental Kafka storage engine to store offsets in Keeper instead of relying on committing them to Kafka. [#57625](https://github.com/ClickHouse/ClickHouse/pull/57625) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add new TimeSeries table engine: - by default: ``` CREATE TABLE tbl ENGINE=TimeSeries ``` - or with specifying engines of its internal tables:. [#64183](https://github.com/ClickHouse/ClickHouse/pull/64183) ([Vitaly Baranov](https://github.com/vitlibar)). +* Support more join strictnesses (`LEFT/RIGHT SEMI/ANTI/ANY JOIN`) with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y (see setting `allow_experimental_join_condition`). [#64281](https://github.com/ClickHouse/ClickHouse/pull/64281) ([lgbo](https://github.com/lgbo-ustc)). +* Add `_etag` virtual column for S3 table engine. Fixes [#65312](https://github.com/ClickHouse/ClickHouse/issues/65312). [#65386](https://github.com/ClickHouse/ClickHouse/pull/65386) ([skyoct](https://github.com/skyoct)). +* This pull request introduces Hive-style partitioning for different engines (`File`, `URL`, `S3`, `AzureBlobStorage`, `HDFS`). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). [#65997](https://github.com/ClickHouse/ClickHouse/pull/65997) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Add function printf for spark compatiability. [#66257](https://github.com/ClickHouse/ClickHouse/pull/66257) ([李扬](https://github.com/taiyang-li)). +* Backported in [#68450](https://github.com/ClickHouse/ClickHouse/issues/68450): Implement new JSON data type. [#66444](https://github.com/ClickHouse/ClickHouse/pull/66444) ([Kruglov Pavel](https://github.com/Avogar)). +* Add a new server setting: disable_insertion_and_mutation Set it to true. This node will deny all insertions and mutations(Alter table delete/update/drop partition). Include async insertion. [#66519](https://github.com/ClickHouse/ClickHouse/pull/66519) ([Xu Jia](https://github.com/XuJia0210)). +* Add options `restore_replace_external_engines_to_null` and `restore_replace_external_table_functions_to_null` to replace external engines and table_engines to Null engine that can be useful for testing. It should work for RESTORE and explicit table creation. [#66536](https://github.com/ClickHouse/ClickHouse/pull/66536) ([Ilya Yatsishin](https://github.com/qoega)). +* Added support for reading MULTILINESTRING geometry in WKT format using function readWKTLineString. [#67647](https://github.com/ClickHouse/ClickHouse/pull/67647) ([Jacob Reckhard](https://github.com/jacobrec)). +* Add a new table function `fuzzQuery`. This function allows the modification of a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;`. [#67655](https://github.com/ClickHouse/ClickHouse/pull/67655) ([pufit](https://github.com/pufit)). +* Support query `DROP DETACHED PARTITION ALL` to drop all detached partitions. [#67885](https://github.com/ClickHouse/ClickHouse/pull/67885) ([Duc Canh Le](https://github.com/canhld94)). +* Added a tagging (namespace) mechanism for the query cache. The same queries with different tags are considered different by the query cache. Example: `SELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'abc'` and `SELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'def'` now create different query cache entries. [#68235](https://github.com/ClickHouse/ClickHouse/pull/68235) ([sakulali](https://github.com/sakulali)). + +#### Performance Improvement +* Use adaptive read task size calculation method (adaptive meaning it depends on read column sizes) for parallel replicas. [#60377](https://github.com/ClickHouse/ClickHouse/pull/60377) ([Nikita Taranov](https://github.com/nickitat)). +* Store the `plain_rewritable` disk directory metadata in `__meta` layout, separately from the merge tree data in the object storage. Move the `plain_rewritable` disk to a flat directory structure. [#65751](https://github.com/ClickHouse/ClickHouse/pull/65751) ([Julia Kartseva](https://github.com/jkartseva)). +* Enable `compile_expressions` (JIT compiler for fragments of ordinary expressions) by default. This closes [#51264](https://github.com/ClickHouse/ClickHouse/issues/51264) and [#56386](https://github.com/ClickHouse/ClickHouse/issues/56386). [#66486](https://github.com/ClickHouse/ClickHouse/pull/66486) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve columns squashing for String/Array/Map/Variant/Dynamic types by reserving required memory in advance for all subcolumns. [#67043](https://github.com/ClickHouse/ClickHouse/pull/67043) ([Kruglov Pavel](https://github.com/Avogar)). +* Speed up system flush logs, flush logs on shutdown. [#67472](https://github.com/ClickHouse/ClickHouse/pull/67472) ([Sema Checherinda](https://github.com/CheSema)). +* Backported in [#68496](https://github.com/ClickHouse/ClickHouse/issues/68496): Improved overall performance of merges by reducing the overhead of scheduling steps of merges. [#68016](https://github.com/ClickHouse/ClickHouse/pull/68016) ([Anton Popov](https://github.com/CurtizJ)). +* Setting `optimize_functions_to_subcolumns` is enabled by default. [#68053](https://github.com/ClickHouse/ClickHouse/pull/68053) ([Anton Popov](https://github.com/CurtizJ)). + +#### Improvement +* ClickHouse server now supports new setting `max_keep_alive_requests`. For keep-alive HTTP connections to the server it works in tandem with `keep_alive_timeout` - if idle timeout not expired but there already more than `max_keep_alive_requests` requests done through the given connection - it will be closed by the server. [#61793](https://github.com/ClickHouse/ClickHouse/pull/61793) ([Nikita Taranov](https://github.com/nickitat)). +* As in the new version, SOURCES are checked based on Table Engine logic, even grant table engine is disabled by default, if a source is not granted, a prompt of table engine would popup instead, which is misleading. [#65419](https://github.com/ClickHouse/ClickHouse/pull/65419) ([jsc0218](https://github.com/jsc0218)). +* Added statistics type `count_min` (count-min sketches) which provide selectivity estimations for equality predicates like `col = 'val'`. Supported data types are string, date, datatime and numeric types. [#65521](https://github.com/ClickHouse/ClickHouse/pull/65521) ([JackyWoo](https://github.com/JackyWoo)). +* Do not pass logs for keeper explicitly in the image to allow overriding. [#65564](https://github.com/ClickHouse/ClickHouse/pull/65564) ([Azat Khuzhin](https://github.com/azat)). +* Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#65860](https://github.com/ClickHouse/ClickHouse/pull/65860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add the `rows_before_aggregation_at_least` statistic to the query response when `rows_before_aggregation` is enabled. This statistic represents the number of rows read before aggregation. In the context of a distributed query, when using the `group by` or `max` aggregation function without a `limit`, `rows_before_aggregation_at_least` can reflect the number of rows hit by the query. [#66084](https://github.com/ClickHouse/ClickHouse/pull/66084) ([morning-color](https://github.com/morning-color)). +* Introduced `use_same_password_for_base_backup` settings for `BACKUP` and `RESTORE` queries, allowing to create and restore incremental backups to/from password protected archives. [#66214](https://github.com/ClickHouse/ClickHouse/pull/66214) ([Samuele](https://github.com/sguerrini97)). +* Ignore async_load_databases for ATTACH query (previously it was possible for ATTACH to return before the tables had been attached). [#66240](https://github.com/ClickHouse/ClickHouse/pull/66240) ([Azat Khuzhin](https://github.com/azat)). +* [Replicated]MergeTreeSink has to properly cancel its delayed_chunk on `onCancel()` method. [#66279](https://github.com/ClickHouse/ClickHouse/pull/66279) ([Sema Checherinda](https://github.com/CheSema)). +* Added logs and metrics for rejected connections (where there are not enough resources). [#66410](https://github.com/ClickHouse/ClickHouse/pull/66410) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Setting `allow_experimental_analyzer` is renamed to `enable_analyzer`. The old name is preserved in a form of an alias. [#66438](https://github.com/ClickHouse/ClickHouse/pull/66438) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support true UUID type for MongoDB engine. [#66671](https://github.com/ClickHouse/ClickHouse/pull/66671) ([Azat Khuzhin](https://github.com/azat)). +* Added a new `MergeTree` setting `deduplicate_merge_projection_mode` to control the projections during merges (for specific engines) and `OPTIMIZE DEDUPLICATE` query. Supported options: `throw` (throw an exception in case the projection is not fully supported for *MergeTree engine), `drop` (remove projection during merge if it can't be merged itself consistently) and `rebuild` (rebuild projection from scratch, which is a heavy operation). [#66672](https://github.com/ClickHouse/ClickHouse/pull/66672) ([jsc0218](https://github.com/jsc0218)). +* Add replication lag and recovery time metrics. [#66703](https://github.com/ClickHouse/ClickHouse/pull/66703) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add S3DiskNoKeyErrors metric. [#66704](https://github.com/ClickHouse/ClickHouse/pull/66704) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Ensure COMMENT clause works for all table engines. [#66832](https://github.com/ClickHouse/ClickHouse/pull/66832) ([Joe Lynch](https://github.com/joelynch)). +* Update the usage of error code `BAD_ARGUMENTS` and `ILLEGAL_TYPE_OF_ARGUMENT` by more accurate error codes when appropriate. [#66851](https://github.com/ClickHouse/ClickHouse/pull/66851) ([Yohann Jardin](https://github.com/yohannj)). +* Function `mapFromArrays` now accepts `Map(K, V)` as first argument, for example: `SELECT mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb'])` now works and returns `{('a',4):'aa',('b',4):'bb'}`. Also, if the 1st argument is an Array, it can now also be of type `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as long as the actual array values are not `NULL`. [#67103](https://github.com/ClickHouse/ClickHouse/pull/67103) ([李扬](https://github.com/taiyang-li)). +* Read configuration for clickhouse-local from ~/.clickhouse-local. [#67135](https://github.com/ClickHouse/ClickHouse/pull/67135) ([Azat Khuzhin](https://github.com/azat)). +* Rename setting `input_format_orc_read_use_writer_time_zone` to `input_format_orc_reader_timezone` and allow the user to set the reader timezone. [#67175](https://github.com/ClickHouse/ClickHouse/pull/67175) ([kevinyhzou](https://github.com/KevinyhZou)). +* Decrease level of 'Socket is not connected' error when HTTP connection immediately reset by peer after connecting, close [#34218](https://github.com/ClickHouse/ClickHouse/issues/34218). [#67177](https://github.com/ClickHouse/ClickHouse/pull/67177) ([vdimir](https://github.com/vdimir)). +* Speed up tables removal for `DROP DATABASE` query, increased the default value for `database_catalog_drop_table_concurrency` to 16. [#67228](https://github.com/ClickHouse/ClickHouse/pull/67228) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add ability to load dashboards for system.dashboards from config (once set, they overrides the default dashboards preset). [#67232](https://github.com/ClickHouse/ClickHouse/pull/67232) ([Azat Khuzhin](https://github.com/azat)). +* The window functions in SQL are traditionally in snake case. ClickHouse uses camelCase, so new aliases `denseRank()` and `percentRank()` have been created. These new functions can be called the exact same as the original `dense_rank()` and `percent_rank()` functions. Both snake case and camelCase syntaxes remain usable. A new test for each of the functions has been added as well. This closes [#67042](https://github.com/ClickHouse/ClickHouse/issues/67042) . [#67334](https://github.com/ClickHouse/ClickHouse/pull/67334) ([Peter Nguyen](https://github.com/petern48)). +* Autodetect configuration file format if is not .xml, .yml or .yaml. If the file begins with < it might be XML, otherwise it might be YAML. Non regular file just parse as XML such as PIPE: /dev/fd/X. [#67391](https://github.com/ClickHouse/ClickHouse/pull/67391) ([sakulali](https://github.com/sakulali)). +* Functions `formatDateTime` and `formatDateTimeInJodaSyntax` now treat their format parameter as optional. If it is not specified, format strings `%Y-%m-%d %H:%i:%s` and `yyyy-MM-dd HH:mm:ss` are assumed. Example: `SELECT parseDateTime('2021-01-04 23:12:34')` now returns DateTime value `2021-01-04 23:12:34` (previously, this threw an exception). [#67399](https://github.com/ClickHouse/ClickHouse/pull/67399) ([Robert Schulze](https://github.com/rschu1ze)). +* Automatically retry Keeper requests in KeeperMap if they happen because of timeout or connection loss. [#67448](https://github.com/ClickHouse/ClickHouse/pull/67448) ([Antonio Andelic](https://github.com/antonio2368)). +* Rework usage of custom table's disks. [#67684](https://github.com/ClickHouse/ClickHouse/pull/67684) ([Sema Checherinda](https://github.com/CheSema)). +* Various improvements in the advanced dashboard. This closes [#67697](https://github.com/ClickHouse/ClickHouse/issues/67697). This closes [#63407](https://github.com/ClickHouse/ClickHouse/issues/63407). This closes [#51129](https://github.com/ClickHouse/ClickHouse/issues/51129). This closes [#61204](https://github.com/ClickHouse/ClickHouse/issues/61204). [#67701](https://github.com/ClickHouse/ClickHouse/pull/67701) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Avoid allocate too much capacity for array column while writing orc & some minor refactors to make code cleaner. Performance speeds up 15% for array column. [#67879](https://github.com/ClickHouse/ClickHouse/pull/67879) ([李扬](https://github.com/taiyang-li)). +* Support OPTIMIZE query on Join table engine to reduce Join tables memory footprint. [#67883](https://github.com/ClickHouse/ClickHouse/pull/67883) ([Duc Canh Le](https://github.com/canhld94)). +* Add replication lag and recovery time metrics. [#67913](https://github.com/ClickHouse/ClickHouse/pull/67913) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add '-no-pie' to aarch64 Linux builds to allow proper introspection and symbolizing of stacktraces after a ClickHouse restart. [#67916](https://github.com/ClickHouse/ClickHouse/pull/67916) ([filimonov](https://github.com/filimonov)). +* Backported in [#68481](https://github.com/ClickHouse/ClickHouse/issues/68481): Added profile events for merges and mutations for better introspection. [#68015](https://github.com/ClickHouse/ClickHouse/pull/68015) ([Anton Popov](https://github.com/CurtizJ)). +* Fix settings/current_database in system.processes for async BACKUP/RESTORE. [#68163](https://github.com/ClickHouse/ClickHouse/pull/68163) ([Azat Khuzhin](https://github.com/azat)). +* Remove unnecessary logs for MergeTree that doesn't support replication. [#68238](https://github.com/ClickHouse/ClickHouse/pull/68238) ([Daniil Ivanik](https://github.com/divanik)). +* Backported in [#68430](https://github.com/ClickHouse/ClickHouse/issues/68430): Improve schema inference of date times. Now DateTime64 used only when date time has fractional part, otherwise regular DateTime is used. Inference of Date/DateTime is more strict now, especially when `date_time_input_format='best_effort'` to avoid inferring date times from strings in corner cases. [#68382](https://github.com/ClickHouse/ClickHouse/pull/68382) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Fixed reading of subcolumns after `ALTER ADD COLUMN` query. [#66243](https://github.com/ClickHouse/ClickHouse/pull/66243) ([Anton Popov](https://github.com/CurtizJ)). +* Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Fix formatting of query with aliased JOIN ON expression, e.g. `... JOIN t2 ON (x = y) AS e ORDER BY x` should be formatted as `... JOIN t2 ON ((x = y) AS e) ORDER BY x`. [#66312](https://github.com/ClickHouse/ClickHouse/pull/66312) ([vdimir](https://github.com/vdimir)). +* Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible runtime error while converting Array field with nulls to Array(Variant). [#66727](https://github.com/ClickHouse/ClickHouse/pull/66727) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Fix creating KeeperMap table after an incomplete drop. [#66865](https://github.com/ClickHouse/ClickHouse/pull/66865) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix broken part error while restoring to a `s3_plain_rewritable` disk. [#66881](https://github.com/ClickHouse/ClickHouse/pull/66881) ([Vitaly Baranov](https://github.com/vitlibar)). +* In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Fix invalid format detection in schema inference that could lead to logical error Format {} doesn't support schema inference. [#66899](https://github.com/ClickHouse/ClickHouse/pull/66899) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible deadlock on query cancel with parallel replicas. [#66905](https://github.com/ClickHouse/ClickHouse/pull/66905) ([Nikita Taranov](https://github.com/nickitat)). +* Forbid create as select even when database_replicated_allow_heavy_create is set. It was unconditionally forbidden in 23.12 and accidentally allowed under the setting in unreleased 24.7. [#66980](https://github.com/ClickHouse/ClickHouse/pull/66980) ([vdimir](https://github.com/vdimir)). +* Reading from the `numbers` could wrongly throw an exception when the `max_rows_to_read` limit was set. This closes [#66992](https://github.com/ClickHouse/ClickHouse/issues/66992). [#66996](https://github.com/ClickHouse/ClickHouse/pull/66996) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add proper type conversion to lagInFrame and leadInFrame window functions - fixes msan test. [#67091](https://github.com/ClickHouse/ClickHouse/pull/67091) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Use a separate client context in `clickhouse-local`. [#67133](https://github.com/ClickHouse/ClickHouse/pull/67133) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Correct behavior of `ORDER BY all` with disabled `enable_order_by_all` and parallel replicas (distributed queries as well). [#67153](https://github.com/ClickHouse/ClickHouse/pull/67153) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix wrong usage of input_format_max_bytes_to_read_for_schema_inference in schema cache. [#67157](https://github.com/ClickHouse/ClickHouse/pull/67157) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix the memory leak for count distinct, when exception issued during group by single nullable key. [#67171](https://github.com/ClickHouse/ClickHouse/pull/67171) ([Jet He](https://github.com/compasses)). +* This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix error `Conversion from AggregateFunction(name, Type) to AggregateFunction(name, Nullable(Type)) is not supported`. The bug was caused by the `optimize_rewrite_aggregate_function_with_if` optimization. Fixes [#67112](https://github.com/ClickHouse/ClickHouse/issues/67112). [#67229](https://github.com/ClickHouse/ClickHouse/pull/67229) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix hung query when using empty tuple as lhs of function IN. [#67295](https://github.com/ClickHouse/ClickHouse/pull/67295) ([Duc Canh Le](https://github.com/canhld94)). +* Fix crash of `uniq` and `uniqTheta ` with `tuple()` argument. Closes [#67303](https://github.com/ClickHouse/ClickHouse/issues/67303). [#67306](https://github.com/ClickHouse/ClickHouse/pull/67306) ([flynn](https://github.com/ucasfl)). +* It was possible to create a very deep nested JSON data that triggered stack overflow while skipping unknown fields. This closes [#67292](https://github.com/ClickHouse/ClickHouse/issues/67292). [#67324](https://github.com/ClickHouse/ClickHouse/pull/67324) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix attaching ReplicatedMergeTree table after exception during startup. [#67360](https://github.com/ClickHouse/ClickHouse/pull/67360) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix segfault caused by incorrectly detaching from thread group in `Aggregator`. [#67385](https://github.com/ClickHouse/ClickHouse/pull/67385) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix one more case when a non-deterministic function is specified in PK. [#67395](https://github.com/ClickHouse/ClickHouse/pull/67395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `bloom_filter` index breaking queries with mildly weird conditions like `(k=2)=(k=2)` or `has([1,2,3], k)`. [#67423](https://github.com/ClickHouse/ClickHouse/pull/67423) ([Michael Kolupaev](https://github.com/al13n321)). +* Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix wait for tasks in ~WriteBufferFromS3 in case WriteBuffer was cancelled. [#67459](https://github.com/ClickHouse/ClickHouse/pull/67459) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Protect temporary part directories from removing during RESTORE. [#67491](https://github.com/ClickHouse/ClickHouse/pull/67491) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Fix `Logical error: Expected the argument №N of type T to have X rows, but it has 0`. The error could happen in a remote query with constant expression in `GROUP BY` (with a new analyzer). [#67536](https://github.com/ClickHouse/ClickHouse/pull/67536) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix join on tuple with NULLs: Some queries with the new analyzer and `NULL` inside the tuple in the `JOIN ON` section returned incorrect results. [#67538](https://github.com/ClickHouse/ClickHouse/pull/67538) ([vdimir](https://github.com/vdimir)). +* Fix redundant reschedule of FileCache::freeSpaceRatioKeepingThreadFunc() in case of full non-evictable cache. [#67540](https://github.com/ClickHouse/ClickHouse/pull/67540) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix for function `toStartOfWeek` which returned the wrong result with a small `DateTime64` value. [#67558](https://github.com/ClickHouse/ClickHouse/pull/67558) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix `Logical error: 'file_offset_of_buffer_end <= read_until_position'` in filesystem cache. Closes [#57508](https://github.com/ClickHouse/ClickHouse/issues/57508). [#67623](https://github.com/ClickHouse/ClickHouse/pull/67623) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixes [#62282](https://github.com/ClickHouse/ClickHouse/issues/62282). Removed the call to `convertFieldToString()` and added datatype specific serialization code. Parameterized view substitution was broken for multiple datatypes when parameter value was a function or expression returning datatype instance. [#67654](https://github.com/ClickHouse/ClickHouse/pull/67654) ([Shankar](https://github.com/shiyer7474)). +* Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix possible logical error "Unexpected return type from if" with experimental Variant type and enabled setting `use_variant_as_common_type ` in function if with Tuples and Maps. [#67687](https://github.com/ClickHouse/ClickHouse/pull/67687) ([Kruglov Pavel](https://github.com/Avogar)). +* Due to a bug in Linux Kernel, a query can hung in `TimerDescriptor::drain`. This closes [#37686](https://github.com/ClickHouse/ClickHouse/issues/37686). [#67702](https://github.com/ClickHouse/ClickHouse/pull/67702) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix completion of `RESTORE ON CLUSTER` command. [#67720](https://github.com/ClickHouse/ClickHouse/pull/67720) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading. [#67751](https://github.com/ClickHouse/ClickHouse/pull/67751) ([Azat Khuzhin](https://github.com/azat)). +* Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Queries like `SELECT count() FROM t WHERE cast(c = 1 or c = 9999 AS Bool) SETTINGS use_skip_indexes=1` with bloom filter indexes on `c` now work correctly. [#67781](https://github.com/ClickHouse/ClickHouse/pull/67781) ([jsc0218](https://github.com/jsc0218)). +* Fix wrong aggregation result in some queries with aggregation without keys and filter, close [#67419](https://github.com/ClickHouse/ClickHouse/issues/67419). [#67804](https://github.com/ClickHouse/ClickHouse/pull/67804) ([vdimir](https://github.com/vdimir)). +* Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix DateTime64 parsing after constant folding in distributed queries, close [#66773](https://github.com/ClickHouse/ClickHouse/issues/66773). [#67920](https://github.com/ClickHouse/ClickHouse/pull/67920) ([vdimir](https://github.com/vdimir)). +* Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Now ClickHouse doesn't consider part as broken if projection doesn't exist on disk but exists in `checksums.txt`. [#68003](https://github.com/ClickHouse/ClickHouse/pull/68003) ([alesapin](https://github.com/alesapin)). +* Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Attempt to fix `Block structure mismatch in AggregatingStep stream: different types` for aggregate projection optimization. [#68107](https://github.com/ClickHouse/ClickHouse/pull/68107) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68343](https://github.com/ClickHouse/ClickHouse/issues/68343): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68400](https://github.com/ClickHouse/ClickHouse/issues/68400): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). + +#### Build/Testing/Packaging Improvement +* Improved `test_storage_s3` tests: increased `s3_max_single_read_retries` for read from "unstable" s3 source and allowed all tests to run multiple times in a row. [#66896](https://github.com/ClickHouse/ClickHouse/pull/66896) ([Ilya Yatsishin](https://github.com/qoega)). +* Integration tests flaky check will not run each test case multiple times to find more issues in tests and make them more reliable. It is using `pytest-repeat` library to run test case multiple times for the same environment. It is important to cleanup tables and other entities in the end of a test case to pass. Repeat works much faster than several pytest runs as it starts necessary containers only once. [#66986](https://github.com/ClickHouse/ClickHouse/pull/66986) ([Ilya Yatsishin](https://github.com/qoega)). +* Allow to use CLion with ClickHouse. In previous versions, CLion freezed for a minute on every keypress. This closes [#66994](https://github.com/ClickHouse/ClickHouse/issues/66994). [#66995](https://github.com/ClickHouse/ClickHouse/pull/66995) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Getauxval: avoid crash under sanitizer re-exec due to high aslr entropy. [#67081](https://github.com/ClickHouse/ClickHouse/pull/67081) ([Raúl Marín](https://github.com/Algunenano)). +* Some parts of client code are extracted to a single file and highest possible level optimization is applied to them even for debug builds. This closes: [#65745](https://github.com/ClickHouse/ClickHouse/issues/65745). [#67215](https://github.com/ClickHouse/ClickHouse/pull/67215) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### NO CL CATEGORY + +* Backported in [#68416](https://github.com/ClickHouse/ClickHouse/issues/68416):. [#68386](https://github.com/ClickHouse/ClickHouse/pull/68386) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Fix for 992 and friends"'. [#66993](https://github.com/ClickHouse/ClickHouse/pull/66993) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Fix for 992 and friends""'. [#67029](https://github.com/ClickHouse/ClickHouse/pull/67029) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "FuzzQuery table function"'. [#67040](https://github.com/ClickHouse/ClickHouse/pull/67040) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Enable `compile_expressions` by default."'. [#67299](https://github.com/ClickHouse/ClickHouse/pull/67299) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Slightly better calculation of primary index"'. [#67392](https://github.com/ClickHouse/ClickHouse/pull/67392) ([alesapin](https://github.com/alesapin)). +* NO CL ENTRY: 'Revert "Add settings to replace external engines to Null during create"'. [#67507](https://github.com/ClickHouse/ClickHouse/pull/67507) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Add settings to replace external engines to Null during create""'. [#67511](https://github.com/ClickHouse/ClickHouse/pull/67511) ([Ilya Yatsishin](https://github.com/qoega)). +* NO CL ENTRY: 'Revert "Add replication lag and recovery time metrics"'. [#67731](https://github.com/ClickHouse/ClickHouse/pull/67731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Slightly better calculation of primary index""'. [#67846](https://github.com/ClickHouse/ClickHouse/pull/67846) ([Anton Popov](https://github.com/CurtizJ)). +* NO CL ENTRY: 'Revert "CI: Strict job timeout 1.5h for tests, 2h for builds"'. [#67986](https://github.com/ClickHouse/ClickHouse/pull/67986) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing"'. [#68014](https://github.com/ClickHouse/ClickHouse/pull/68014) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* NO CL ENTRY: 'Revert "Use `Atomic` database by default in `clickhouse-local`"'. [#68023](https://github.com/ClickHouse/ClickHouse/pull/68023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Refactor tests for (experimental) statistics"'. [#68156](https://github.com/ClickHouse/ClickHouse/pull/68156) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* CI: enable libfuzzer (fixing build and docker). [#61908](https://github.com/ClickHouse/ClickHouse/pull/61908) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Initial implementation of vector similarity index. [#63675](https://github.com/ClickHouse/ClickHouse/pull/63675) ([Robert Schulze](https://github.com/rschu1ze)). +* Update zlib-ng from 2.0.2 to 2.1.7. [#64489](https://github.com/ClickHouse/ClickHouse/pull/64489) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix 02444_async_broken_outdated_part_loading flakiness. [#64956](https://github.com/ClickHouse/ClickHouse/pull/64956) ([Azat Khuzhin](https://github.com/azat)). +* attach_gdb.lib: print more information before all stacks. [#65253](https://github.com/ClickHouse/ClickHouse/pull/65253) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix some perf tests. [#65320](https://github.com/ClickHouse/ClickHouse/pull/65320) ([Nikita Taranov](https://github.com/nickitat)). +* Remove ActionsDAGPtr whenever it is possible. [#65414](https://github.com/ClickHouse/ClickHouse/pull/65414) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Scheduler queue throughput introspection, Fix CPU indication in client. [#65654](https://github.com/ClickHouse/ClickHouse/pull/65654) ([Sergei Trifonov](https://github.com/serxa)). +* Increase timeout in 02122_join_group_by_timeout for tsan build. [#65976](https://github.com/ClickHouse/ClickHouse/pull/65976) ([vdimir](https://github.com/vdimir)). +* Remove default values for certificateFile/privateKeyFile/dhParamsFile in keeper config (to avoid annoying errors in logs). [#65978](https://github.com/ClickHouse/ClickHouse/pull/65978) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v24.3.5.46-lts. [#66054](https://github.com/ClickHouse/ClickHouse/pull/66054) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix flaky `02814_currentDatabase_for_table_functions`. [#66111](https://github.com/ClickHouse/ClickHouse/pull/66111) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix possible data-race StorageKafka with statistics_interval_ms>0. [#66311](https://github.com/ClickHouse/ClickHouse/pull/66311) ([Azat Khuzhin](https://github.com/azat)). +* Avoid unneeded calculation in SeriesPeriodDetect. [#66320](https://github.com/ClickHouse/ClickHouse/pull/66320) ([Ruihang Xia](https://github.com/waynexia)). +* It aims to complete [#58630](https://github.com/ClickHouse/ClickHouse/issues/58630). This is made possible by [#60463](https://github.com/ClickHouse/ClickHouse/issues/60463), [#61459](https://github.com/ClickHouse/ClickHouse/issues/61459) and [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#66443](https://github.com/ClickHouse/ClickHouse/pull/66443) ([Amos Bird](https://github.com/amosbird)). +* Allow run query instantly in play. [#66457](https://github.com/ClickHouse/ClickHouse/pull/66457) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Bump ICU from v70 to v75. [#66474](https://github.com/ClickHouse/ClickHouse/pull/66474) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump RocksDB from v6.29.5 to v7.10.2. [#66475](https://github.com/ClickHouse/ClickHouse/pull/66475) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump RocksDB from v7.10.2 to v8.9.1. [#66479](https://github.com/ClickHouse/ClickHouse/pull/66479) ([Robert Schulze](https://github.com/rschu1ze)). +* I believe the error code for this function should not be "NOT_ALLOWED" since it simply is an invalid query and "BAD_QUERY_PARAMETER" is a more reasonable error code for this. [#66491](https://github.com/ClickHouse/ClickHouse/pull/66491) ([Ali](https://github.com/xogoodnow)). +* Update gdb to 15.1 (by compiling from sources). [#66494](https://github.com/ClickHouse/ClickHouse/pull/66494) ([Azat Khuzhin](https://github.com/azat)). +* Ensure that llvm-symbolizer is used for symbolizing sanitizer reports. [#66495](https://github.com/ClickHouse/ClickHouse/pull/66495) ([Azat Khuzhin](https://github.com/azat)). +* Remove unused local variables. [#66503](https://github.com/ClickHouse/ClickHouse/pull/66503) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This will solve a lot of problems with inconsistent formatting. And it opens the path for [#65753](https://github.com/ClickHouse/ClickHouse/issues/65753). This closes [#66807](https://github.com/ClickHouse/ClickHouse/issues/66807). This closes [#61611](https://github.com/ClickHouse/ClickHouse/issues/61611). This closes [#61711](https://github.com/ClickHouse/ClickHouse/issues/61711). This closes [#67445](https://github.com/ClickHouse/ClickHouse/issues/67445). [#66506](https://github.com/ClickHouse/ClickHouse/pull/66506) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Rename Context::getSettings() to Context::getSettingsCopy(). [#66528](https://github.com/ClickHouse/ClickHouse/pull/66528) ([Raúl Marín](https://github.com/Algunenano)). +* Uninteresting change: introducing `ClientApplicationBase`. [#66549](https://github.com/ClickHouse/ClickHouse/pull/66549) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Respond to parallel replicas protocol requests with priority on initiator. [#66618](https://github.com/ClickHouse/ClickHouse/pull/66618) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix bad code: it was catching exceptions. [#66628](https://github.com/ClickHouse/ClickHouse/pull/66628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Dump all memory stats in CgroupsMemoryUsageObserver on hitting the limit. [#66732](https://github.com/ClickHouse/ClickHouse/pull/66732) ([Nikita Taranov](https://github.com/nickitat)). +* Save writer thread id in shared mutex for debugging. [#66745](https://github.com/ClickHouse/ClickHouse/pull/66745) ([Alexander Gololobov](https://github.com/davenger)). +* Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Looks like it runs too many mutations sometimes and fails to process them within the timeout. So if a query waits for mutations - the test fails. [#66785](https://github.com/ClickHouse/ClickHouse/pull/66785) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Better diagnostics in functional tests. [#66790](https://github.com/ClickHouse/ClickHouse/pull/66790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `01042_system_reload_dictionary_reloads_completely`. [#66811](https://github.com/ClickHouse/ClickHouse/pull/66811) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Something is strange with the test about refreshable materialized views. [#66816](https://github.com/ClickHouse/ClickHouse/pull/66816) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Randomize `trace_profile_events`. [#66821](https://github.com/ClickHouse/ClickHouse/pull/66821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Stop ignoring SIGSEGV in GDB. [#66847](https://github.com/ClickHouse/ClickHouse/pull/66847) ([Antonio Andelic](https://github.com/antonio2368)). +* Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* When executing queries with parallel replicas that involve only a subset of nodes within a shard, the current behavior is that if all participating replicas are unavailable, the query completes without any errors but returns no results. Referencing issue [#65467](https://github.com/ClickHouse/ClickHouse/issues/65467), this pull request addresses the issue where only a portion of the nodes in a shard are participating in the execution. [#66880](https://github.com/ClickHouse/ClickHouse/pull/66880) ([zoomxi](https://github.com/zoomxi)). +* Speed up stateful tests setup. [#66886](https://github.com/ClickHouse/ClickHouse/pull/66886) ([Raúl Marín](https://github.com/Algunenano)). +* Functions [h-r]*: Iterate over input_rows_count where appropriate. [#66897](https://github.com/ClickHouse/ClickHouse/pull/66897) ([Robert Schulze](https://github.com/rschu1ze)). +* Stateless tests: Change status of failed tests in case of server crash and add no-parallel to high-load tests. [#66901](https://github.com/ClickHouse/ClickHouse/pull/66901) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix performance test about the generateRandom table function, supposedly. [#66906](https://github.com/ClickHouse/ClickHouse/pull/66906) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad tests `share_big_sets`, CC @davenger. [#66908](https://github.com/ClickHouse/ClickHouse/pull/66908) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Speed up mutations for non-replicated MergeTree a bit. [#66909](https://github.com/ClickHouse/ClickHouse/pull/66909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Speed up mutations for non-replicated MergeTree significantly. [#66911](https://github.com/ClickHouse/ClickHouse/pull/66911) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix views over distributed tables with Analyzer. [#66912](https://github.com/ClickHouse/ClickHouse/pull/66912) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [CI fest] Try to fix `test_broken_projections/test.py::test_broken_ignored_replicated`. [#66915](https://github.com/ClickHouse/ClickHouse/pull/66915) ([Andrey Zvonov](https://github.com/zvonand)). +* Decrease rate limit in `01923_network_receive_time_metric_insert`. [#66924](https://github.com/ClickHouse/ClickHouse/pull/66924) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Grouparrayintersect: fix serialization bug. [#66928](https://github.com/ClickHouse/ClickHouse/pull/66928) ([Raúl Marín](https://github.com/Algunenano)). +* Update version after release branch. [#66929](https://github.com/ClickHouse/ClickHouse/pull/66929) ([Raúl Marín](https://github.com/Algunenano)). +* Un-flake test_runtime_configurable_cache_size. [#66934](https://github.com/ClickHouse/ClickHouse/pull/66934) ([Robert Schulze](https://github.com/rschu1ze)). +* fix unit tests ResolvePoolTest with timeouts. [#66953](https://github.com/ClickHouse/ClickHouse/pull/66953) ([Sema Checherinda](https://github.com/CheSema)). +* Split slow test 03036_dynamic_read_subcolumns. [#66954](https://github.com/ClickHouse/ClickHouse/pull/66954) ([Nikita Taranov](https://github.com/nickitat)). +* CI: Fixes docker server build for release branches. [#66955](https://github.com/ClickHouse/ClickHouse/pull/66955) ([Max K.](https://github.com/maxknv)). +* Addressing issue [#64936](https://github.com/ClickHouse/ClickHouse/issues/64936). [#66973](https://github.com/ClickHouse/ClickHouse/pull/66973) ([alesapin](https://github.com/alesapin)). +* Add initial 24.7 changelog. [#66976](https://github.com/ClickHouse/ClickHouse/pull/66976) ([Raúl Marín](https://github.com/Algunenano)). +* Apply libunwind fix. [#66977](https://github.com/ClickHouse/ClickHouse/pull/66977) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Add logs for debugging. [#66979](https://github.com/ClickHouse/ClickHouse/pull/66979) ([Max K.](https://github.com/maxknv)). +* [CI Fest] Split dynamic tests and rewrite them from sh to sql to avoid timeouts. [#66981](https://github.com/ClickHouse/ClickHouse/pull/66981) ([Kruglov Pavel](https://github.com/Avogar)). +* Split 01508_partition_pruning_long. [#66983](https://github.com/ClickHouse/ClickHouse/pull/66983) ([Nikita Taranov](https://github.com/nickitat)). +* [CI Fest] Fix use-of-uninitialized-value in JSONExtract* numeric functions. [#66984](https://github.com/ClickHouse/ClickHouse/pull/66984) ([Kruglov Pavel](https://github.com/Avogar)). +* It should fix SQLancer checks, but for some reason we stopped invalidating cache for docker builds and fix was not published to our CI for a while. [#66987](https://github.com/ClickHouse/ClickHouse/pull/66987) ([Ilya Yatsishin](https://github.com/qoega)). +* Fixes [#66941](https://github.com/ClickHouse/ClickHouse/issues/66941). [#66991](https://github.com/ClickHouse/ClickHouse/pull/66991) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Remove the support for Kerberized HDFS. [#66998](https://github.com/ClickHouse/ClickHouse/pull/66998) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Fix for workflow results parsing. [#67000](https://github.com/ClickHouse/ClickHouse/pull/67000) ([Max K.](https://github.com/maxknv)). +* Fix flaky `01454_storagememory_data_race_challenge`. [#67003](https://github.com/ClickHouse/ClickHouse/pull/67003) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: Jepsen Workflow fix for skipped builds and observability. [#67004](https://github.com/ClickHouse/ClickHouse/pull/67004) ([Max K.](https://github.com/maxknv)). +* bugfix AttachedTable counting not symmetry, and adding some test logs…. [#67007](https://github.com/ClickHouse/ClickHouse/pull/67007) ([Xu Jia](https://github.com/XuJia0210)). +* CI: Automerge when required and non-required checks completed. [#67008](https://github.com/ClickHouse/ClickHouse/pull/67008) ([Max K.](https://github.com/maxknv)). +* Fix test `very_long_arrays`. [#67009](https://github.com/ClickHouse/ClickHouse/pull/67009) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try to fix exception logging in destructors of static objects. [#67016](https://github.com/ClickHouse/ClickHouse/pull/67016) ([Antonio Andelic](https://github.com/antonio2368)). +* [Green CI] Fix test test_storage_azure_blob_storage. [#67019](https://github.com/ClickHouse/ClickHouse/pull/67019) ([Daniil Ivanik](https://github.com/divanik)). +* Integration tests: fix flaky tests `test_backup_restore_on_cluster/test_concurrency.py` & `test_manipulate_statistics/test.py`. [#67027](https://github.com/ClickHouse/ClickHouse/pull/67027) ([Nikita Fomichev](https://github.com/fm4v)). +* [Green CI] Fix test test_storage_s3_queue/test.py::test_max_set_age. [#67035](https://github.com/ClickHouse/ClickHouse/pull/67035) ([Pablo Marcos](https://github.com/pamarcos)). +* Test for alter select with parallel replicas. [#67041](https://github.com/ClickHouse/ClickHouse/pull/67041) ([Igor Nikonov](https://github.com/devcrafter)). +* Split query into multiple queries to consume less memory at once + use less data. Fixes [#67034](https://github.com/ClickHouse/ClickHouse/issues/67034). [#67044](https://github.com/ClickHouse/ClickHouse/pull/67044) ([alesapin](https://github.com/alesapin)). +* Disable setting `optimize_functions_to_subcolumns`. [#67046](https://github.com/ClickHouse/ClickHouse/pull/67046) ([Anton Popov](https://github.com/CurtizJ)). +* Increase max allocation size for sanitizers. [#67049](https://github.com/ClickHouse/ClickHouse/pull/67049) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* rewrite 01171 test. [#67054](https://github.com/ClickHouse/ClickHouse/pull/67054) ([Sema Checherinda](https://github.com/CheSema)). +* Add `**` to `hdfs` docs, add test for `**` in `hdfs`. [#67064](https://github.com/ClickHouse/ClickHouse/pull/67064) ([Andrey Zvonov](https://github.com/zvonand)). +* Very sad failure: ``` 2024.07.24 13:28:45.517777 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} executeQuery: (from 172.16.11.1:55890) OPTIMIZE TABLE replicated_mt FINAL (stage: Complete) 2024.07.24 13:28:45.525945 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (ReplicatedMergeTreeQueue): Waiting for 4 entries to be processed: queue-0000000004, queue-0000000002, queue-0000000001, queue-0000000000 2024.07.24 13:29:15.528024 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e) (MergerMutator): Selected 3 parts from all_0_0_0 to all_2_2_0 2024.07.24 13:29:15.530736 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Created log entry /clickhouse/tables/replicated_mt/log/log-0000000004 for merge all_0_2_1 2024.07.24 13:29:15.530873 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to process log entry 2024.07.24 13:29:15.530919 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to pull log-0000000004 to queue 2024.07.24 13:29:15.534286 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Looking for node corresponding to log-0000000004 in node1 queue 2024.07.24 13:29:15.534793 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for queue-0000000005 to disappear from node1 queue 2024.07.24 13:29:15.585533 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} TCPHandler: Processed in 30.067804125 sec. ```. [#67067](https://github.com/ClickHouse/ClickHouse/pull/67067) ([alesapin](https://github.com/alesapin)). +* Fix flaky `test_seekable_formats_url` and `test_seekable_formats` S3 storage tests. [#67070](https://github.com/ClickHouse/ClickHouse/pull/67070) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* CI: Docker server build fix for new release workflow. [#67075](https://github.com/ClickHouse/ClickHouse/pull/67075) ([Max K.](https://github.com/maxknv)). +* Fix 2680 flasky. [#67078](https://github.com/ClickHouse/ClickHouse/pull/67078) ([jsc0218](https://github.com/jsc0218)). +* [CI Fest] Fix flaky 02447_drop_replica test. [#67085](https://github.com/ClickHouse/ClickHouse/pull/67085) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fixes [#67030](https://github.com/ClickHouse/ClickHouse/issues/67030). [#67086](https://github.com/ClickHouse/ClickHouse/pull/67086) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Increase timeout for curl in tests. [#67089](https://github.com/ClickHouse/ClickHouse/pull/67089) ([Anton Popov](https://github.com/CurtizJ)). +* Try calculating available memory if ClickHouse is bound to subset of NUMA nodes. [#67098](https://github.com/ClickHouse/ClickHouse/pull/67098) ([Antonio Andelic](https://github.com/antonio2368)). +* A more precise way of tracking flushing time in 01246_buffer_flush. [#67099](https://github.com/ClickHouse/ClickHouse/pull/67099) ([Azat Khuzhin](https://github.com/azat)). +* Do not fail CheckReadyForMerge on failed Tests_2 (non-required jobs) Do not skip CiBuddy report step on failures. [#67101](https://github.com/ClickHouse/ClickHouse/pull/67101) ([Max K.](https://github.com/maxknv)). +* Tststs_1 - for all required checks Tststs_2 - for all non-required checks (normal mode) Tststs_2_ww - for all non-required checks (woolenwolfdog mode). [#67104](https://github.com/ClickHouse/ClickHouse/pull/67104) ([Max K.](https://github.com/maxknv)). +* Functions [s-t]*: Iterate over input_rows_count where appropriate. [#67105](https://github.com/ClickHouse/ClickHouse/pull/67105) ([Robert Schulze](https://github.com/rschu1ze)). +* Reintroduce 02805_distributed_queries_timeouts. [#67106](https://github.com/ClickHouse/ClickHouse/pull/67106) ([Azat Khuzhin](https://github.com/azat)). +* Added some tests in relation with [#54881](https://github.com/ClickHouse/ClickHouse/issues/54881). [#67110](https://github.com/ClickHouse/ClickHouse/pull/67110) ([max-vostrikov](https://github.com/max-vostrikov)). +* Reintroduce 03002_part_log_rmt_fetch_*_error tests without flakiness and less time. [#67113](https://github.com/ClickHouse/ClickHouse/pull/67113) ([Azat Khuzhin](https://github.com/azat)). +* Improve tag matching in backport scripts. [#67118](https://github.com/ClickHouse/ClickHouse/pull/67118) ([Raúl Marín](https://github.com/Algunenano)). +* Fixes [#67111](https://github.com/ClickHouse/ClickHouse/issues/67111). [#67121](https://github.com/ClickHouse/ClickHouse/pull/67121) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Increase lock_acquire_timeout_for_background_operations setting in dynamic merges tests. [#67126](https://github.com/ClickHouse/ClickHouse/pull/67126) ([Kruglov Pavel](https://github.com/Avogar)). +* Attempt to fix flakiness of some window view tests. [#67130](https://github.com/ClickHouse/ClickHouse/pull/67130) ([Robert Schulze](https://github.com/rschu1ze)). +* Update assert in cache. [#67138](https://github.com/ClickHouse/ClickHouse/pull/67138) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test `00673_subquery_prepared_set_performance`. [#67141](https://github.com/ClickHouse/ClickHouse/pull/67141) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes [#67047](https://github.com/ClickHouse/ClickHouse/issues/67047). [#67142](https://github.com/ClickHouse/ClickHouse/pull/67142) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Lower max allocation size in query fuzzer. [#67145](https://github.com/ClickHouse/ClickHouse/pull/67145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes [#66966](https://github.com/ClickHouse/ClickHouse/issues/66966). [#67147](https://github.com/ClickHouse/ClickHouse/pull/67147) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Try fix `02481_async_insert_race_long` flakiness. [#67148](https://github.com/ClickHouse/ClickHouse/pull/67148) ([Julia Kartseva](https://github.com/jkartseva)). +* Rename (unreleased) bad setting. [#67149](https://github.com/ClickHouse/ClickHouse/pull/67149) ([Raúl Marín](https://github.com/Algunenano)). +* Uncomment accidentally commented out code in QueryProfiler. [#67152](https://github.com/ClickHouse/ClickHouse/pull/67152) ([Michael Kolupaev](https://github.com/al13n321)). +* Try to fix 2572. [#67158](https://github.com/ClickHouse/ClickHouse/pull/67158) ([jsc0218](https://github.com/jsc0218)). +* Fix benign data race in ZooKeeper. [#67164](https://github.com/ClickHouse/ClickHouse/pull/67164) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove too long unit test. [#67168](https://github.com/ClickHouse/ClickHouse/pull/67168) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `00705_drop_create_merge_tree`. [#67170](https://github.com/ClickHouse/ClickHouse/pull/67170) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix stacktrace cache. [#67173](https://github.com/ClickHouse/ClickHouse/pull/67173) ([Antonio Andelic](https://github.com/antonio2368)). +* Fixes [#67151](https://github.com/ClickHouse/ClickHouse/issues/67151). [#67174](https://github.com/ClickHouse/ClickHouse/pull/67174) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Make 02908_many_requests_to_system_replicas less stressful. [#67176](https://github.com/ClickHouse/ClickHouse/pull/67176) ([Alexander Gololobov](https://github.com/davenger)). +* Reduce max time of 00763_long_lock_buffer_alter_destination_table. [#67185](https://github.com/ClickHouse/ClickHouse/pull/67185) ([Raúl Marín](https://github.com/Algunenano)). +* Do not count AttachedTable for tables in information schema databases. [#67187](https://github.com/ClickHouse/ClickHouse/pull/67187) ([Sergei Trifonov](https://github.com/serxa)). +* Verbose output for 03203_client_benchmark_options. [#67188](https://github.com/ClickHouse/ClickHouse/pull/67188) ([vdimir](https://github.com/vdimir)). +* Split test 02967_parallel_replicas_join_algo_and_analyzer. [#67211](https://github.com/ClickHouse/ClickHouse/pull/67211) ([Nikita Taranov](https://github.com/nickitat)). +* Fix flaky `test_pkill_query_log` (tsan). [#67223](https://github.com/ClickHouse/ClickHouse/pull/67223) ([Sergei Trifonov](https://github.com/serxa)). +* Remove integration test `test_broken_projections_in_backups_1`. [#67231](https://github.com/ClickHouse/ClickHouse/pull/67231) ([Vitaly Baranov](https://github.com/vitlibar)). +* Debug logging for [#67002](https://github.com/ClickHouse/ClickHouse/issues/67002). [#67233](https://github.com/ClickHouse/ClickHouse/pull/67233) ([Nikita Taranov](https://github.com/nickitat)). +* Fix oss-fuzz build. [#67235](https://github.com/ClickHouse/ClickHouse/pull/67235) ([Nikita Taranov](https://github.com/nickitat)). +* Fix flaky 00180_no_seek_avoiding_when_reading_from_cache. [#67236](https://github.com/ClickHouse/ClickHouse/pull/67236) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* English. [#67258](https://github.com/ClickHouse/ClickHouse/pull/67258) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove strange code. [#67260](https://github.com/ClickHouse/ClickHouse/pull/67260) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MSan report in DatabaseReplicated. [#67262](https://github.com/ClickHouse/ClickHouse/pull/67262) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02310_clickhouse_local_INSERT_progress_profile_events`. [#67264](https://github.com/ClickHouse/ClickHouse/pull/67264) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove test `02982_aggregation_states_destruction`. [#67266](https://github.com/ClickHouse/ClickHouse/pull/67266) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix OOM in test runs. [#67268](https://github.com/ClickHouse/ClickHouse/pull/67268) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove harmful stuff from tests. [#67275](https://github.com/ClickHouse/ClickHouse/pull/67275) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `03201_variant_null_map_subcolumn`. [#67276](https://github.com/ClickHouse/ClickHouse/pull/67276) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Split `01651_lc_insert_tiny_log`. [#67279](https://github.com/ClickHouse/ClickHouse/pull/67279) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Debug test `02490_benchmark_max_consecutive_errors`. [#67281](https://github.com/ClickHouse/ClickHouse/pull/67281) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02833_concurrrent_sessions`. [#67282](https://github.com/ClickHouse/ClickHouse/pull/67282) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a separate test for exception handling. [#67283](https://github.com/ClickHouse/ClickHouse/pull/67283) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Debug test `01600_parts_states_metrics_long`. [#67284](https://github.com/ClickHouse/ClickHouse/pull/67284) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Faster test `02231_buffer_aggregate_states_leak`. [#67285](https://github.com/ClickHouse/ClickHouse/pull/67285) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix curiosities in `TimerDescriptor`. [#67287](https://github.com/ClickHouse/ClickHouse/pull/67287) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add retries to test `02911_backup_restore_keeper_map`. [#67290](https://github.com/ClickHouse/ClickHouse/pull/67290) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Functions: Iterate over input_rows_count where appropriate. [#67294](https://github.com/ClickHouse/ClickHouse/pull/67294) ([Robert Schulze](https://github.com/rschu1ze)). +* Add documentation for `compile_expressions`. [#67300](https://github.com/ClickHouse/ClickHouse/pull/67300) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Wrap log lines in the CI report for functional tests. [#67301](https://github.com/ClickHouse/ClickHouse/pull/67301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02050_client_profile_events`. [#67309](https://github.com/ClickHouse/ClickHouse/pull/67309) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* I do not want to think about this code. [#67312](https://github.com/ClickHouse/ClickHouse/pull/67312) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `00940_max_parts_in_total`. [#67313](https://github.com/ClickHouse/ClickHouse/pull/67313) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tests for Kafka cannot run in parallel. [#67315](https://github.com/ClickHouse/ClickHouse/pull/67315) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#66285](https://github.com/ClickHouse/ClickHouse/issues/66285). [#67325](https://github.com/ClickHouse/ClickHouse/pull/67325) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Follow-up for [#67301](https://github.com/ClickHouse/ClickHouse/issues/67301). [#67327](https://github.com/ClickHouse/ClickHouse/pull/67327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#61659](https://github.com/ClickHouse/ClickHouse/issues/61659). [#67332](https://github.com/ClickHouse/ClickHouse/pull/67332) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix integration test `test_backup_restore_on_cluster/test_disallow_concurrency`. [#67336](https://github.com/ClickHouse/ClickHouse/pull/67336) ([Vitaly Baranov](https://github.com/vitlibar)). +* Faster and less flaky 01246_buffer_flush (by using HTTP over clickhouse-client). [#67340](https://github.com/ClickHouse/ClickHouse/pull/67340) ([Azat Khuzhin](https://github.com/azat)). +* Fix: data race in TCPHandler on socket timeouts settings. [#67341](https://github.com/ClickHouse/ClickHouse/pull/67341) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* CI: Print stdout, stderr for docker pull command. [#67343](https://github.com/ClickHouse/ClickHouse/pull/67343) ([Max K.](https://github.com/maxknv)). +* Followup [#67290](https://github.com/ClickHouse/ClickHouse/issues/67290). [#67348](https://github.com/ClickHouse/ClickHouse/pull/67348) ([vdimir](https://github.com/vdimir)). +* Skip parallel for `test_storage_kerberized_kafka`. [#67349](https://github.com/ClickHouse/ClickHouse/pull/67349) ([Andrey Zvonov](https://github.com/zvonand)). +* Don't use PeekableReadBuffer in JSONAsObject format. [#67354](https://github.com/ClickHouse/ClickHouse/pull/67354) ([Kruglov Pavel](https://github.com/Avogar)). +* This closes: [#57316](https://github.com/ClickHouse/ClickHouse/issues/57316). [#67355](https://github.com/ClickHouse/ClickHouse/pull/67355) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Remove duplicated tests. [#67357](https://github.com/ClickHouse/ClickHouse/pull/67357) ([Kruglov Pavel](https://github.com/Avogar)). +* Release branch was not detected properly and job which must run on release branch could be reused from feature branches. PR Fixes detection of release branches. [#67358](https://github.com/ClickHouse/ClickHouse/pull/67358) ([Max K.](https://github.com/maxknv)). +* Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Add no-distributed-cache tag in tests. [#67361](https://github.com/ClickHouse/ClickHouse/pull/67361) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Trying to fix test_cache_evicted_by_temporary_data and print debug info. [#67362](https://github.com/ClickHouse/ClickHouse/pull/67362) ([vdimir](https://github.com/vdimir)). +* Try to fix: ALL_CONNECTION_TRIES_FAILED with parallel replicas. [#67389](https://github.com/ClickHouse/ClickHouse/pull/67389) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix bad test `01036_no_superfluous_dict_reload_on_create_database`. [#67390](https://github.com/ClickHouse/ClickHouse/pull/67390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Adding `SelectedPartsTotal` and `SelectedMarksTotal` as new ProfileEvents. [#67393](https://github.com/ClickHouse/ClickHouse/pull/67393) ([Jordi Villar](https://github.com/jrdi)). +* Print debug info in `test_storage_s3_queue/test.py::test_shards_distributed`. [#67394](https://github.com/ClickHouse/ClickHouse/pull/67394) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Mute degraded perf test. [#67396](https://github.com/ClickHouse/ClickHouse/pull/67396) ([Nikita Taranov](https://github.com/nickitat)). +* Debug TimerDescriptor. [#67397](https://github.com/ClickHouse/ClickHouse/pull/67397) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove has_single_port property from plan stream. [#67398](https://github.com/ClickHouse/ClickHouse/pull/67398) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix typo. [#67400](https://github.com/ClickHouse/ClickHouse/pull/67400) ([Halersson Paris](https://github.com/halersson)). +* CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). +* Disable 02932_refreshable_materialized_views. [#67404](https://github.com/ClickHouse/ClickHouse/pull/67404) ([Michael Kolupaev](https://github.com/al13n321)). +* Follow-up to [#67294](https://github.com/ClickHouse/ClickHouse/issues/67294). [#67405](https://github.com/ClickHouse/ClickHouse/pull/67405) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix DWARF range list parsing in stack symbolizer. [#67417](https://github.com/ClickHouse/ClickHouse/pull/67417) ([Michael Kolupaev](https://github.com/al13n321)). +* Make Dwarf::findAddress() fallback slow path less slow. [#67418](https://github.com/ClickHouse/ClickHouse/pull/67418) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix inconsistent formatting of CODEC and STATISTICS. [#67421](https://github.com/ClickHouse/ClickHouse/pull/67421) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Reduced complexity of the test 02832_alter_max_sessions_for_user. [#67425](https://github.com/ClickHouse/ClickHouse/pull/67425) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Remove obsolete `--multiquery` parameter from tests. [#67435](https://github.com/ClickHouse/ClickHouse/pull/67435) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix public backports. [#67439](https://github.com/ClickHouse/ClickHouse/pull/67439) ([Raúl Marín](https://github.com/Algunenano)). +* Bump Azure from v1.12 to v1.13. [#67446](https://github.com/ClickHouse/ClickHouse/pull/67446) ([Robert Schulze](https://github.com/rschu1ze)). +* 24.7 add missing documentation and testing. [#67454](https://github.com/ClickHouse/ClickHouse/pull/67454) ([Nikita Fomichev](https://github.com/fm4v)). +* Use correct order of fields in `StorageURLSource`. [#67455](https://github.com/ClickHouse/ClickHouse/pull/67455) ([Antonio Andelic](https://github.com/antonio2368)). +* run 01171 test in parallel. [#67470](https://github.com/ClickHouse/ClickHouse/pull/67470) ([Sema Checherinda](https://github.com/CheSema)). +* [Green CI] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown. [#67474](https://github.com/ClickHouse/ClickHouse/pull/67474) ([Alexey Katsman](https://github.com/alexkats)). +* Reduce 02473_multistep_prewhere run time. [#67475](https://github.com/ClickHouse/ClickHouse/pull/67475) ([Alexander Gololobov](https://github.com/davenger)). +* Update version_date.tsv and changelogs after v24.7.1.2915-stable. [#67483](https://github.com/ClickHouse/ClickHouse/pull/67483) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Revert [#61750](https://github.com/ClickHouse/ClickHouse/issues/61750) "Improve JSONEachRow reading by ignoring the keys case". [#67484](https://github.com/ClickHouse/ClickHouse/pull/67484) ([Michael Kolupaev](https://github.com/al13n321)). +* Disable parallel run for `01923_network_receive_time_metric_insert.sh`. [#67492](https://github.com/ClickHouse/ClickHouse/pull/67492) ([Julia Kartseva](https://github.com/jkartseva)). +* Fix test `test_backup_restore_on_cluster/test.py::test_mutation`. [#67494](https://github.com/ClickHouse/ClickHouse/pull/67494) ([Vitaly Baranov](https://github.com/vitlibar)). +* [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. [#67506](https://github.com/ClickHouse/ClickHouse/pull/67506) ([Alexey Katsman](https://github.com/alexkats)). +* [Green CI] Test `test_storage_azure_blob_storage/test.py` is flaky. [#67512](https://github.com/ClickHouse/ClickHouse/pull/67512) ([Daniil Ivanik](https://github.com/divanik)). +* Prepare Release workflow for production. [#67523](https://github.com/ClickHouse/ClickHouse/pull/67523) ([Max K.](https://github.com/maxknv)). +* Fix upgrade check. [#67524](https://github.com/ClickHouse/ClickHouse/pull/67524) ([Raúl Marín](https://github.com/Algunenano)). +* [Green CI] test 03164_s3_settings_for_queries_and_merges is flaky. [#67535](https://github.com/ClickHouse/ClickHouse/pull/67535) ([Daniil Ivanik](https://github.com/divanik)). +* Log message and increased concurrency for table removal. [#67537](https://github.com/ClickHouse/ClickHouse/pull/67537) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix AsyncLoader destruction race. [#67553](https://github.com/ClickHouse/ClickHouse/pull/67553) ([Sergei Trifonov](https://github.com/serxa)). +* Add an assert into TimerDescriptor. [#67555](https://github.com/ClickHouse/ClickHouse/pull/67555) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Re-enable ICU on s390/x. [#67557](https://github.com/ClickHouse/ClickHouse/pull/67557) ([Robert Schulze](https://github.com/rschu1ze)). +* Update version_date.tsv and changelogs after v24.4.4.107-stable. [#67559](https://github.com/ClickHouse/ClickHouse/pull/67559) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Sometimes we fail with timeout in stateless tests and the reason for that seems to be in `stop_logs_replication` step. Add a check for timeout here. [#67560](https://github.com/ClickHouse/ClickHouse/pull/67560) ([Nikolay Degterinsky](https://github.com/evillique)). +* Miscellaneous. [#67564](https://github.com/ClickHouse/ClickHouse/pull/67564) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* enable parallel_view_processing in perf tests. [#67565](https://github.com/ClickHouse/ClickHouse/pull/67565) ([Sema Checherinda](https://github.com/CheSema)). +* Fix flaky `test_system_kafka_consumers_rebalance`. [#67566](https://github.com/ClickHouse/ClickHouse/pull/67566) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Update version_date.tsv and changelogs after v24.7.2.13-stable. [#67586](https://github.com/ClickHouse/ClickHouse/pull/67586) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix 01811_storage_buffer_flush_parameters flakiness. [#67589](https://github.com/ClickHouse/ClickHouse/pull/67589) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. [#67590](https://github.com/ClickHouse/ClickHouse/pull/67590) ([Pablo Marcos](https://github.com/pamarcos)). +* Update minio in integration tests. [#67595](https://github.com/ClickHouse/ClickHouse/pull/67595) ([Antonio Andelic](https://github.com/antonio2368)). +* added tests for page index in parquet files. [#67596](https://github.com/ClickHouse/ClickHouse/pull/67596) ([max-vostrikov](https://github.com/max-vostrikov)). +* Update check_rabbitmq_is_available. [#67597](https://github.com/ClickHouse/ClickHouse/pull/67597) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix 02434_cancel_insert_when_client_dies. [#67600](https://github.com/ClickHouse/ClickHouse/pull/67600) ([vdimir](https://github.com/vdimir)). +* Fix 02910_bad_logs_level_in_local in fast tests. [#67603](https://github.com/ClickHouse/ClickHouse/pull/67603) ([Raúl Marín](https://github.com/Algunenano)). +* Fix 01605_adaptive_granularity_block_borders. [#67605](https://github.com/ClickHouse/ClickHouse/pull/67605) ([Nikita Taranov](https://github.com/nickitat)). +* Update CHANGELOG.md. [#67607](https://github.com/ClickHouse/ClickHouse/pull/67607) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove some `no-parallel` tags from tests. [#67610](https://github.com/ClickHouse/ClickHouse/pull/67610) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md. [#67613](https://github.com/ClickHouse/ClickHouse/pull/67613) ([Tyler Hannan](https://github.com/tylerhannan)). +* Try fix 03143_asof_join_ddb_long. [#67620](https://github.com/ClickHouse/ClickHouse/pull/67620) ([Nikita Taranov](https://github.com/nickitat)). +* Don't run ASAN unit tests under gdb. [#67622](https://github.com/ClickHouse/ClickHouse/pull/67622) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash in KeyCondition::cloneASTWithInversionPushDown() caused by type change. [#67641](https://github.com/ClickHouse/ClickHouse/pull/67641) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix race condition between ProcessList and Settings. [#67645](https://github.com/ClickHouse/ClickHouse/pull/67645) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `02481_async_insert_race_long.sh` flakiness fixes. [#67650](https://github.com/ClickHouse/ClickHouse/pull/67650) ([Julia Kartseva](https://github.com/jkartseva)). +* Fixes [#67651](https://github.com/ClickHouse/ClickHouse/issues/67651). [#67653](https://github.com/ClickHouse/ClickHouse/pull/67653) ([pufit](https://github.com/pufit)). +* Fix flaky `test_replicated_table_attach`. [#67658](https://github.com/ClickHouse/ClickHouse/pull/67658) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v24.4.4.113-stable. [#67659](https://github.com/ClickHouse/ClickHouse/pull/67659) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Even better healthcheck for ldap. [#67667](https://github.com/ClickHouse/ClickHouse/pull/67667) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix 03203_client_benchmark_options. [#67671](https://github.com/ClickHouse/ClickHouse/pull/67671) ([vdimir](https://github.com/vdimir)). +* Integration tests: fix ports clashing problem. [#67672](https://github.com/ClickHouse/ClickHouse/pull/67672) ([Nikita Fomichev](https://github.com/fm4v)). +* Remove some `no-parallel` tags from tests (Part 2). [#67673](https://github.com/ClickHouse/ClickHouse/pull/67673) ([Raúl Marín](https://github.com/Algunenano)). +* Use FunctionArgumentDescriptors for bitSlice. [#67674](https://github.com/ClickHouse/ClickHouse/pull/67674) ([Lennard Eijsackers](https://github.com/Blokje5)). +* Update version_date.tsv and changelog after v24.3.6.48-lts. [#67677](https://github.com/ClickHouse/ClickHouse/pull/67677) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Avoid ddl queue timeout in 02313_filesystem_cache_seeks. [#67680](https://github.com/ClickHouse/ClickHouse/pull/67680) ([Nikita Taranov](https://github.com/nickitat)). +* Fix bad log message in sort description. [#67690](https://github.com/ClickHouse/ClickHouse/pull/67690) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelog after v23.8.16.40-lts. [#67692](https://github.com/ClickHouse/ClickHouse/pull/67692) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix check names in test reports and the CI Logs database. [#67696](https://github.com/ClickHouse/ClickHouse/pull/67696) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Minor refactoring in ci_utils. [#67706](https://github.com/ClickHouse/ClickHouse/pull/67706) ([Max K.](https://github.com/maxknv)). +* Fix 01042_system_reload_dictionary_reloads_completely flakiness. [#67719](https://github.com/ClickHouse/ClickHouse/pull/67719) ([Azat Khuzhin](https://github.com/azat)). +* Fix test `00002_log_and_exception_messages_formatting`. [#67723](https://github.com/ClickHouse/ClickHouse/pull/67723) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02789_reading_from_s3_with_connection_pool`. [#67726](https://github.com/ClickHouse/ClickHouse/pull/67726) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix strange code in HostResolvePool. [#67727](https://github.com/ClickHouse/ClickHouse/pull/67727) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix a typo. [#67729](https://github.com/ClickHouse/ClickHouse/pull/67729) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Smart handling of processes leftovers in tests. [#67737](https://github.com/ClickHouse/ClickHouse/pull/67737) ([Azat Khuzhin](https://github.com/azat)). +* Fix test retries. [#67738](https://github.com/ClickHouse/ClickHouse/pull/67738) ([Azat Khuzhin](https://github.com/azat)). +* Fill only selected columns from system.clusters. [#67739](https://github.com/ClickHouse/ClickHouse/pull/67739) ([Azat Khuzhin](https://github.com/azat)). +* Bump NuRaft (to properly catch thread exceptions). [#67740](https://github.com/ClickHouse/ClickHouse/pull/67740) ([Azat Khuzhin](https://github.com/azat)). +* Try to fix RabbitMQ test failures. [#67743](https://github.com/ClickHouse/ClickHouse/pull/67743) ([Azat Khuzhin](https://github.com/azat)). +* Stateless tests: attempt to fix timeouts of `02473_multistep_prewhere* 00411_long_accurate_number_comparison*`. [#67746](https://github.com/ClickHouse/ClickHouse/pull/67746) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix test_ttl_move::test_alter_with_merge_work flakiness. [#67747](https://github.com/ClickHouse/ClickHouse/pull/67747) ([Azat Khuzhin](https://github.com/azat)). +* ci: better stateless runner (correctly collect artifacts and also some basic errors capturing). [#67752](https://github.com/ClickHouse/ClickHouse/pull/67752) ([Azat Khuzhin](https://github.com/azat)). +* Introduce `no-flaky-check` tag. [#67755](https://github.com/ClickHouse/ClickHouse/pull/67755) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Whitespaces. [#67771](https://github.com/ClickHouse/ClickHouse/pull/67771) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [RFC] Print original query for AST formatting check on CI. [#67776](https://github.com/ClickHouse/ClickHouse/pull/67776) ([Azat Khuzhin](https://github.com/azat)). +* Fix test `02833_concurrent_sessions`, Fix test `02835_drop_user_during_session`. [#67779](https://github.com/ClickHouse/ClickHouse/pull/67779) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix diagnostics in the test script. [#67780](https://github.com/ClickHouse/ClickHouse/pull/67780) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02231_bloom_filter_sizing`. [#67784](https://github.com/ClickHouse/ClickHouse/pull/67784) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed session_log related tests race condition on logout. [#67785](https://github.com/ClickHouse/ClickHouse/pull/67785) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* ci/stateless: fix artifacts post-processing and detect if something failed there. [#67791](https://github.com/ClickHouse/ClickHouse/pull/67791) ([Azat Khuzhin](https://github.com/azat)). +* Integration tests: fix flaky `test_dictionaries_update_and_reload::test_reload_after_fail_by_timer`. [#67793](https://github.com/ClickHouse/ClickHouse/pull/67793) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix possible CANNOT_READ_ALL_DATA during server startup in performance tests. [#67795](https://github.com/ClickHouse/ClickHouse/pull/67795) ([Azat Khuzhin](https://github.com/azat)). +* Reduce table size in 03037_dynamic_merges_2* tests. [#67797](https://github.com/ClickHouse/ClickHouse/pull/67797) ([Kruglov Pavel](https://github.com/Avogar)). +* Disable 03038_nested_dynamic_merges* under sanitizers because it's too slow. [#67798](https://github.com/ClickHouse/ClickHouse/pull/67798) ([Kruglov Pavel](https://github.com/Avogar)). +* Revert "Merge pull request [#66510](https://github.com/ClickHouse/ClickHouse/issues/66510) from canhld94/fix_trivial_count_non_deterministic_func". [#67800](https://github.com/ClickHouse/ClickHouse/pull/67800) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Update comment. [#67801](https://github.com/ClickHouse/ClickHouse/pull/67801) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix bad test `03032_redundant_equals`. [#67822](https://github.com/ClickHouse/ClickHouse/pull/67822) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update tags for a few tests. [#67829](https://github.com/ClickHouse/ClickHouse/pull/67829) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add debug logging for window view tests. [#67841](https://github.com/ClickHouse/ClickHouse/pull/67841) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Closes [#67621](https://github.com/ClickHouse/ClickHouse/issues/67621). [#67843](https://github.com/ClickHouse/ClickHouse/pull/67843) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix query cache randomization in stress tests. [#67855](https://github.com/ClickHouse/ClickHouse/pull/67855) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v24.5.5.78-stable. [#67863](https://github.com/ClickHouse/ClickHouse/pull/67863) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Un-flake 02524_fuzz_and_fuss_2. [#67867](https://github.com/ClickHouse/ClickHouse/pull/67867) ([Robert Schulze](https://github.com/rschu1ze)). +* Misc fixes. [#67869](https://github.com/ClickHouse/ClickHouse/pull/67869) ([Alexey Katsman](https://github.com/alexkats)). +* Fixes [#67444](https://github.com/ClickHouse/ClickHouse/issues/67444). [#67873](https://github.com/ClickHouse/ClickHouse/pull/67873) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* no-msan 00314_sample_factor_virtual_column. [#67874](https://github.com/ClickHouse/ClickHouse/pull/67874) ([Michael Kolupaev](https://github.com/al13n321)). +* Revert "Revert "Add a test for [#47892](https://github.com/ClickHouse/ClickHouse/issues/47892)"". [#67877](https://github.com/ClickHouse/ClickHouse/pull/67877) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Lazily create invalid files in S3. [#67882](https://github.com/ClickHouse/ClickHouse/pull/67882) ([Antonio Andelic](https://github.com/antonio2368)). +* Do not try to create azure container if not needed. [#67896](https://github.com/ClickHouse/ClickHouse/pull/67896) ([Anton Popov](https://github.com/CurtizJ)). +* CI: Fix for setting Mergeable Check from sync. [#67898](https://github.com/ClickHouse/ClickHouse/pull/67898) ([Max K.](https://github.com/maxknv)). +* Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing. [#67904](https://github.com/ClickHouse/ClickHouse/pull/67904) ([Robert Schulze](https://github.com/rschu1ze)). +* Update version_date.tsv and changelogs after v24.6.3.95-stable. [#67910](https://github.com/ClickHouse/ClickHouse/pull/67910) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Remove some no-parallel tags from tests (Part 3). [#67914](https://github.com/ClickHouse/ClickHouse/pull/67914) ([Raúl Marín](https://github.com/Algunenano)). +* Follow up [#67235](https://github.com/ClickHouse/ClickHouse/issues/67235). [#67917](https://github.com/ClickHouse/ClickHouse/pull/67917) ([Nikita Taranov](https://github.com/nickitat)). +* CI: Changelog: Critical Bug Fix to Bug Fix. [#67919](https://github.com/ClickHouse/ClickHouse/pull/67919) ([Max K.](https://github.com/maxknv)). +* CI: Multi-channel CiBuddy. [#67923](https://github.com/ClickHouse/ClickHouse/pull/67923) ([Max K.](https://github.com/maxknv)). +* more logs to debug logical error from async inserts. [#67928](https://github.com/ClickHouse/ClickHouse/pull/67928) ([Han Fei](https://github.com/hanfei1991)). +* Fix stress test error with TDigest statistics. [#67930](https://github.com/ClickHouse/ClickHouse/pull/67930) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove some no-parallel tags from tests (Part 4). [#67932](https://github.com/ClickHouse/ClickHouse/pull/67932) ([Raúl Marín](https://github.com/Algunenano)). +* Upgrade QPL to v1.6.0. [#67933](https://github.com/ClickHouse/ClickHouse/pull/67933) ([Maria Zhukova](https://github.com/mzhukova)). +* CI: Strict job timeout 1.5h for tests, 2h for builds. [#67934](https://github.com/ClickHouse/ClickHouse/pull/67934) ([Max K.](https://github.com/maxknv)). +* Remove slow tests from fasttest check. [#67941](https://github.com/ClickHouse/ClickHouse/pull/67941) ([Raúl Marín](https://github.com/Algunenano)). +* Fix memory corruption in usearch. [#67942](https://github.com/ClickHouse/ClickHouse/pull/67942) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68547](https://github.com/ClickHouse/ClickHouse/issues/68547): Timeout handling for functional and integration tests, store artifacts and report if timed out - sets 2h default timeout for all jobs. [#67944](https://github.com/ClickHouse/ClickHouse/pull/67944) ([Max K.](https://github.com/maxknv)). +* Unflake 02099_tsv_raw_format.sh. [#67947](https://github.com/ClickHouse/ClickHouse/pull/67947) ([Robert Schulze](https://github.com/rschu1ze)). +* This closes: [#67866](https://github.com/ClickHouse/ClickHouse/issues/67866). [#67950](https://github.com/ClickHouse/ClickHouse/pull/67950) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Change log level of an insignificant message in clickhouse-local. [#67952](https://github.com/ClickHouse/ClickHouse/pull/67952) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Fix flaky `test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed`. [#67959](https://github.com/ClickHouse/ClickHouse/pull/67959) ([Julia Kartseva](https://github.com/jkartseva)). +* tests: fix 03002_part_log_rmt_fetch_merge_error flakiness. [#67960](https://github.com/ClickHouse/ClickHouse/pull/67960) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout of 02310_clickhouse_local_INSERT_progress_profile_events. [#67961](https://github.com/ClickHouse/ClickHouse/pull/67961) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove obsolete `--multiquery` parameter (follow-up to [#63898](https://github.com/ClickHouse/ClickHouse/issues/63898)), pt. III. [#67964](https://github.com/ClickHouse/ClickHouse/pull/67964) ([Robert Schulze](https://github.com/rschu1ze)). +* Update minio in stateless tests. [#67975](https://github.com/ClickHouse/ClickHouse/pull/67975) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: Integration tests uncover some logging. [#67978](https://github.com/ClickHouse/ClickHouse/pull/67978) ([Max K.](https://github.com/maxknv)). +* Fix 03130_convert_outer_join_to_inner_join. [#67980](https://github.com/ClickHouse/ClickHouse/pull/67980) ([vdimir](https://github.com/vdimir)). +* Collect minio audit logs in stateless tests. [#67998](https://github.com/ClickHouse/ClickHouse/pull/67998) ([Antonio Andelic](https://github.com/antonio2368)). +* Remove some no-parallel tags from tests (Part 5). [#68002](https://github.com/ClickHouse/ClickHouse/pull/68002) ([Raúl Marín](https://github.com/Algunenano)). +* Minor fixes in tables.md. [#68004](https://github.com/ClickHouse/ClickHouse/pull/68004) ([Ilya Yatsishin](https://github.com/qoega)). +* Follow up for [#67843](https://github.com/ClickHouse/ClickHouse/issues/67843). [#68007](https://github.com/ClickHouse/ClickHouse/pull/68007) ([Ilya Yatsishin](https://github.com/qoega)). +* Remove unused CLI option. [#68008](https://github.com/ClickHouse/ClickHouse/pull/68008) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02845_threads_count_in_distributed_queries`. [#68011](https://github.com/ClickHouse/ClickHouse/pull/68011) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Pass job timeout into tests config. [#68013](https://github.com/ClickHouse/ClickHouse/pull/68013) ([Nikita Fomichev](https://github.com/fm4v)). +* Add a test for [#57420](https://github.com/ClickHouse/ClickHouse/issues/57420). [#68017](https://github.com/ClickHouse/ClickHouse/pull/68017) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Revert "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing"". [#68021](https://github.com/ClickHouse/ClickHouse/pull/68021) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Fix for filtering jobs in PRs. [#68022](https://github.com/ClickHouse/ClickHouse/pull/68022) ([Max K.](https://github.com/maxknv)). +* Docs: Update 3rd party library guide. [#68027](https://github.com/ClickHouse/ClickHouse/pull/68027) ([Robert Schulze](https://github.com/rschu1ze)). +* Refactor tests for (experimental) statistics. [#68034](https://github.com/ClickHouse/ClickHouse/pull/68034) ([Robert Schulze](https://github.com/rschu1ze)). +* Split `00284_external_aggregation.sql`. [#68037](https://github.com/ClickHouse/ClickHouse/pull/68037) ([Robert Schulze](https://github.com/rschu1ze)). +* Update version_date.tsv and changelog after v24.7.3.42-stable. [#68045](https://github.com/ClickHouse/ClickHouse/pull/68045) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update test_drop_is_lock_free/test.py. [#68051](https://github.com/ClickHouse/ClickHouse/pull/68051) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fixes [#67865](https://github.com/ClickHouse/ClickHouse/issues/67865). [#68054](https://github.com/ClickHouse/ClickHouse/pull/68054) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Disable randomization of `trace_profile_events` in clickhouse-test. [#68058](https://github.com/ClickHouse/ClickHouse/pull/68058) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor CMake cleanup. [#68069](https://github.com/ClickHouse/ClickHouse/pull/68069) ([Robert Schulze](https://github.com/rschu1ze)). +* If the test cluster is overloaded, sometimes simple query execution [can take more time](https://pastila.nl/?00224e71/f017cd6675b52ccc205c81aa62a47de5#8dB4+C4MOdOi3NLV1dc0Fg==) than `Buffer`'s max time to flush. This PR doubles the timeout and allows to skip the check in case of significant latency. [#68072](https://github.com/ClickHouse/ClickHouse/pull/68072) ([pufit](https://github.com/pufit)). +* Fix flaky `02675_profile_events_from_query_log_and_client`. [#68097](https://github.com/ClickHouse/ClickHouse/pull/68097) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix race in `WithRetries`. [#68106](https://github.com/ClickHouse/ClickHouse/pull/68106) ([Antonio Andelic](https://github.com/antonio2368)). +* Add empty cell to reports when time is missing. [#68112](https://github.com/ClickHouse/ClickHouse/pull/68112) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix test `00900_long_parquet_load`. [#68130](https://github.com/ClickHouse/ClickHouse/pull/68130) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* tests: fix 01246_buffer_flush flakiness due to slow trace_log flush. [#68134](https://github.com/ClickHouse/ClickHouse/pull/68134) ([Azat Khuzhin](https://github.com/azat)). +* Only use Field::safeGet - Field::get prone to type punning. [#68135](https://github.com/ClickHouse/ClickHouse/pull/68135) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* tests: attempt to fix 01600_parts_states_metrics_long (by forbid parallel run). [#68136](https://github.com/ClickHouse/ClickHouse/pull/68136) ([Azat Khuzhin](https://github.com/azat)). +* Fix01710 Timeout. [#68138](https://github.com/ClickHouse/ClickHouse/pull/68138) ([jsc0218](https://github.com/jsc0218)). +* Remove the extra cell from reports when it is not necessary. [#68145](https://github.com/ClickHouse/ClickHouse/pull/68145) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Remove "Processing configuration file" message from clickhouse-local. [#68157](https://github.com/ClickHouse/ClickHouse/pull/68157) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 02122_join_group_by_timeout flakiness. [#68160](https://github.com/ClickHouse/ClickHouse/pull/68160) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_cluster_all_replicas`. [#68178](https://github.com/ClickHouse/ClickHouse/pull/68178) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix leftovers. [#68181](https://github.com/ClickHouse/ClickHouse/pull/68181) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `01172_transaction_counters`. [#68182](https://github.com/ClickHouse/ClickHouse/pull/68182) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refactor tests for (experimental) statistics. [#68186](https://github.com/ClickHouse/ClickHouse/pull/68186) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove Log engine from Kafka integration tests. [#68200](https://github.com/ClickHouse/ClickHouse/pull/68200) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Replace segfault in Replicated database with logical error. [#68250](https://github.com/ClickHouse/ClickHouse/pull/68250) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#68423](https://github.com/ClickHouse/ClickHouse/issues/68423): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68374](https://github.com/ClickHouse/ClickHouse/issues/68374): Rename: S3DiskNoKeyErrors -> DiskS3NoSuchKeyErrors. [#68361](https://github.com/ClickHouse/ClickHouse/pull/68361) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#68637](https://github.com/ClickHouse/ClickHouse/issues/68637): Check for invalid regexp in JSON SKIP REGEXP section. [#68451](https://github.com/ClickHouse/ClickHouse/pull/68451) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68485](https://github.com/ClickHouse/ClickHouse/issues/68485): Better inference of date times 2. [#68452](https://github.com/ClickHouse/ClickHouse/pull/68452) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68540](https://github.com/ClickHouse/ClickHouse/issues/68540): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68518](https://github.com/ClickHouse/ClickHouse/issues/68518): Minor update in Dynamic/JSON serializations. [#68459](https://github.com/ClickHouse/ClickHouse/pull/68459) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68558](https://github.com/ClickHouse/ClickHouse/issues/68558): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). +* Backported in [#68576](https://github.com/ClickHouse/ClickHouse/issues/68576): CI: Tidy build timeout from 2h to 3h. [#68567](https://github.com/ClickHouse/ClickHouse/pull/68567) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 95ef8c0de90..8556375d543 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.8.1.2684-lts 2024-08-21 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 From 5965297d8b2f26768fb0ee13a9aeec6d7cada0c9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 21 Aug 2024 16:35:39 +0200 Subject: [PATCH 228/409] add accidentally removed virtual column --- src/Storages/VirtualColumnUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 5b974cb8a22..bbeb9ee6643 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -172,6 +172,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); + add_virtual("_etag", std::make_shared(std::make_shared())); if (context->getSettingsRef().use_hive_partitioning) { From b5dadd778b3db0d28e9f1650503ef3f5de42895e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 21 Aug 2024 17:02:25 +0200 Subject: [PATCH 229/409] temp-commit --- .../02293_http_header_full_summary_without_progress.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh index a08928a773c..b888d977129 100755 --- a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh +++ b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh @@ -20,6 +20,7 @@ then echo "Read rows in summary is not zero" else echo "Read rows in summary is zero!" + echo "${CURL_OUTPUT}" fi # Check that the response code is correct too From e87de3cfcd22870bf7aea3dfaf1607b180b2b1d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Aug 2024 15:19:29 +0000 Subject: [PATCH 230/409] return back virtual columns to distributed tables --- src/Storages/StorageDistributed.cpp | 4 ++++ .../03228_virtual_column_merge_dist.reference | 8 +++++++ .../03228_virtual_column_merge_dist.sql | 24 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03228_virtual_column_merge_dist.reference create mode 100644 tests/queries/0_stateless/03228_virtual_column_merge_dist.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e146e95f89f..c4668159759 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,6 +290,10 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); + /// Add virtual columns from table of storage Merges. + desc.addEphemeral("_database", std::make_shared(std::make_shared()), "The name of database which the row comes from"); + desc.addEphemeral("_table", std::make_shared(std::make_shared()), "The name of table which the row comes from"); + return desc; } diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference b/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference new file mode 100644 index 00000000000..28f00bafdfe --- /dev/null +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference @@ -0,0 +1,8 @@ +1 t_local_1 +2 t_local_2 +1 t_local_1 +2 t_local_2 +1 1 +2 1 +1 1 +2 1 diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql new file mode 100644 index 00000000000..caf00a2e407 --- /dev/null +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS t_local_1; +DROP TABLE IF EXISTS t_local_2; +DROP TABLE IF EXISTS t_merge; +DROP TABLE IF EXISTS t_distr; + +CREATE TABLE t_local_1 (a UInt32) ENGINE = MergeTree ORDER BY a; +CREATE TABLE t_local_2 (a UInt32) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_local_1 VALUES (1); +INSERT INTO t_local_2 VALUES (2); + +CREATE TABLE t_merge AS t_local_1 ENGINE = Merge(currentDatabase(), '^(t_local_1|t_local_2)$'); +CREATE TABLE t_distr AS t_local_1 engine=Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); + +SELECT a, _table FROM t_merge ORDER BY a; +SELECT a, _table FROM t_distr ORDER BY a; + +SELECT a, _database = currentDatabase() FROM t_merge ORDER BY a; +SELECT a, _database = currentDatabase() FROM t_distr ORDER BY a; + +DROP TABLE IF EXISTS t_local_1; +DROP TABLE IF EXISTS t_local_2; +DROP TABLE IF EXISTS t_merge; +DROP TABLE IF EXISTS t_distr; From 0c2ad11af27bb4a914475113652345fed1b8d6b1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Aug 2024 17:49:51 +0200 Subject: [PATCH 231/409] Fix flaky check when all tests are skipped --- tests/docker_scripts/stateless_runner.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/docker_scripts/stateless_runner.sh b/tests/docker_scripts/stateless_runner.sh index 671b1f5ca71..40a63f74a6b 100755 --- a/tests/docker_scripts/stateless_runner.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -80,6 +80,9 @@ fi export IS_FLAKY_CHECK=0 +# Export NUM_TRIES so python scripts will see its value as env variable +export NUM_TRIES + # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then export IS_FLAKY_CHECK=1 From 2e58ac56111a075bdbaee566a4484a193a882792 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 21 Aug 2024 16:30:42 +0000 Subject: [PATCH 232/409] build fix --- src/Storages/VirtualColumnUtils.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index bbeb9ee6643..7e3e902f083 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -156,16 +156,20 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto auto add_virtual = [&](const auto & name, const auto & type) { - auto local_type = type; - if (storage_columns.has(name) && !context->getSettingsRef().use_hive_partitioning) - return; if (storage_columns.has(name)) { - local_type = storage_columns.get(name).type; + if (!context->getSettingsRef().use_hive_partitioning) + return; + + if (storage_columns.size() == 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot use a file with one column {}, that is ised during hive partitioning", name); + auto local_type = storage_columns.get(name).type; storage_columns.remove(name); + desc.addEphemeral(name, local_type, ""); + return; } - desc.addEphemeral(name, local_type, ""); + desc.addEphemeral(name, type, ""); }; add_virtual("_path", std::make_shared(std::make_shared())); From a52eff299eb49291e2b57f68e5b2874c7704f9d2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 21 Aug 2024 19:43:45 +0200 Subject: [PATCH 233/409] fix tests --- src/Storages/VirtualColumnUtils.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 43 ++---- tests/integration/test_storage_hdfs/test.py | 24 ++- .../03203_hive_style_partitioning.reference | 145 +++++++++--------- .../03203_hive_style_partitioning.sh | 2 - 5 files changed, 100 insertions(+), 116 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 7e3e902f083..ca82a1ce67a 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -162,7 +162,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return; if (storage_columns.size() == 1) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot use a file with one column {}, that is ised during hive partitioning", name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot implement partition by all columns in a file"); auto local_type = storage_columns.get(name).type; storage_columns.remove(name); desc.addEphemeral(name, local_type, ""); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index fbdc7f29f98..637dbd38262 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1518,14 +1518,14 @@ def test_hive_partitioning_with_one_parameter(cluster): ) query = ( - f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " + f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}')" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=path ) ] @@ -1533,14 +1533,14 @@ def test_hive_partitioning_with_one_parameter(cluster): query = ( f"SELECT column2 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + f"blob_path='{path}', format='CSV', structure='{table_format}');" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Gordon"] -def test_hive_partitioning_with_two_parameters(cluster): +def test_hive_partitioning_with_all_parameters(cluster): # type: (ClickHouseCluster) -> None node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 String, column2 String" @@ -1556,35 +1556,14 @@ def test_hive_partitioning_with_two_parameters(cluster): ) query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + f"blob_path='{path}', format='CSV', structure='{table_format}');" ) - assert azure_query( - node, query, settings={"use_hive_partitioning": 1} - ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( - bucket="cont", max_path=path - ) - ] + pattern = r"DB::Exception: Cannot implement partition by all columns in a file" - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" - ) - assert azure_query( - node, query, settings={"use_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] - - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" - ) - assert azure_query( - node, query, settings={"use_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] + with pytest.raises(Exception, match=pattern): + azure_query(node, query, settings={"use_hive_partitioning": 1}) def test_hive_partitioning_without_setting(cluster): @@ -1603,9 +1582,9 @@ def test_hive_partitioning_without_setting(cluster): ) query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + f"blob_path='{path}', format='CSV', structure='{table_format}');" ) pattern = re.compile( r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index fdbf7c5bacb..ad2e7084791 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1263,13 +1263,19 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" r = node1.query( - "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", settings={"use_hive_partitioning": 1}, ) assert r == f"Elizabeth\n" + r = node1.query( + "SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + settings={"use_hive_partitioning": 1}, + ) + assert r == f"Gordon\n" -def test_hive_partitioning_with_two_parameters(started_cluster): + +def test_hive_partitioning_with_all_parameters(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" @@ -1279,11 +1285,13 @@ def test_hive_partitioning_with_two_parameters(started_cluster): == f"Elizabeth\tGordon\n" ) - r = node1.query( - "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"use_hive_partitioning": 1}, - ) - assert r == f"Gordon\n" + pattern = r"DB::Exception: Cannot implement partition by all columns in a file" + + with pytest.raises(QueryRuntimeException, match=pattern): + node1.query( + f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"use_hive_partitioning": 1}, + ) def test_hive_partitioning_without_setting(started_cluster): @@ -1301,7 +1309,7 @@ def test_hive_partitioning_without_setting(started_cluster): with pytest.raises(QueryRuntimeException, match=pattern): node1.query( - f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"use_hive_partitioning": 0}, ) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 12ffd17c102..b5eaef7f51e 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -1,4 +1,14 @@ TESTING THE FILE HIVE PARTITIONING +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth first last Elizabeth Jorge Frank Elizabeth Hunter Moreno Elizabeth @@ -10,25 +20,16 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -Eva Schmidt Elizabeth Schmidt -Samuel Schmidt Elizabeth Schmidt -Eva Schmidt Elizabeth -Samuel Schmidt Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -first last Elizabeth -Jorge Frank Elizabeth -Hunter Moreno Elizabeth -Esther Guzman Elizabeth -Dennis Stephens Elizabeth -Nettie Franklin Elizabeth -Stanley Gibson Elizabeth -Eugenia Greer Elizabeth -Jeffery Delgado Elizabeth -Clara Cross Elizabeth -Elizabeth Gordon Elizabeth +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth 42 2020-01-01 [1,2,3] 42.42 Array(Int64) LowCardinality(Float64) @@ -37,10 +38,20 @@ Array(Int64) LowCardinality(Float64) 4081 2070 2070 -1 -1 b +1 +1 TESTING THE URL PARTITIONING +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth first last Elizabeth Jorge Frank Elizabeth Hunter Moreno Elizabeth @@ -52,26 +63,18 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -Eva Schmidt Elizabeth Schmidt -Samuel Schmidt Elizabeth Schmidt -Eva Schmidt Elizabeth -Samuel Schmidt Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -first last Elizabeth -Jorge Frank Elizabeth -Hunter Moreno Elizabeth -Esther Guzman Elizabeth -Dennis Stephens Elizabeth -Nettie Franklin Elizabeth -Stanley Gibson Elizabeth -Eugenia Greer Elizabeth -Jeffery Delgado Elizabeth -Clara Cross Elizabeth 1 TESTING THE S3 PARTITIONING +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth first last Elizabeth Jorge Frank Elizabeth Hunter Moreno Elizabeth @@ -83,39 +86,35 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -Eva Schmidt Elizabeth Schmidt -Samuel Schmidt Elizabeth Schmidt -Eva Schmidt Elizabeth -Samuel Schmidt Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -Elizabeth Gordon Elizabeth Gordon -Elizabeth Gordon Elizabeth -first last Elizabeth -Jorge Frank Elizabeth -Hunter Moreno Elizabeth -Esther Guzman Elizabeth -Dennis Stephens Elizabeth -Nettie Franklin Elizabeth -Stanley Gibson Elizabeth -Eugenia Greer Elizabeth -Jeffery Delgado Elizabeth -Clara Cross Elizabeth -Elizabeth Gordon Elizabeth +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth OK TESTING THE S3CLUSTER PARTITIONING -first last Elizabeth -Jorge Frank Elizabeth -Hunter Moreno Elizabeth -Esther Guzman Elizabeth -Dennis Stephens Elizabeth -Nettie Franklin Elizabeth -Stanley Gibson Elizabeth -Eugenia Greer Elizabeth -Jeffery Delgado Elizabeth -Clara Cross Elizabeth -Elizabeth Gordon Elizabeth -Eva Schmidt Elizabeth Schmidt -Samuel Schmidt Elizabeth Schmidt -Eva Schmidt Elizabeth -Samuel Schmidt Elizabeth +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth +last Elizabeth +Frank Elizabeth +Moreno Elizabeth +Guzman Elizabeth +Stephens Elizabeth +Franklin Elizabeth +Gibson Elizabeth +Greer Elizabeth +Delgado Elizabeth +Cross Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 5a0bd482985..41b215578f0 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -13,8 +13,6 @@ set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; From f74b580ddaaf0b339b3370b603c901a22b5d0594 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 21 Aug 2024 19:52:02 +0200 Subject: [PATCH 234/409] More sanity checks --- ...02293_http_header_full_summary_without_progress.sh | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh index b888d977129..29d59a8e45b 100755 --- a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh +++ b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh @@ -6,8 +6,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh +# Sanity check to ensure that the server is up and running +for i in {1..10}; do + echo 'SELECT 1' | ${CLICKHOUSE_CURL_COMMAND} -s "${CLICKHOUSE_URL}" --data-binary @- > /dev/null + if [ $? -eq 0 ]; then + break + fi + sleep 1 +done + CURL_OUTPUT=$(echo 'SELECT 1 + sleepEachRow(0.00002) FROM numbers(100000)' | \ - ${CLICKHOUSE_CURL_COMMAND} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0&max_execution_time=1" --data-binary @- 2>&1) + ${CLICKHOUSE_CURL_COMMAND} --max-time 3 -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0&max_execution_time=1" --data-binary @- 2>&1) READ_ROWS=$(echo "${CURL_OUTPUT}" | \ grep 'X-ClickHouse-Summary' | \ From 09dfec99c756067170ac36fe13e4e8948d92427f Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 21 Aug 2024 20:30:13 +0200 Subject: [PATCH 235/409] Lint --- .../02293_http_header_full_summary_without_progress.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh index 29d59a8e45b..0d8a568fef0 100755 --- a/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh +++ b/tests/queries/0_stateless/02293_http_header_full_summary_without_progress.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Sanity check to ensure that the server is up and running -for i in {1..10}; do +for _ in {1..10}; do echo 'SELECT 1' | ${CLICKHOUSE_CURL_COMMAND} -s "${CLICKHOUSE_URL}" --data-binary @- > /dev/null if [ $? -eq 0 ]; then break From 74d8971432158a312777dcfba229513bfd876acc Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 22:06:52 +0300 Subject: [PATCH 236/409] 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 38f9ef6bc95550d727bc56627fd029741e99177c Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Aug 2024 19:08:07 +0000 Subject: [PATCH 237/409] Fix ColumnVariant permutation --- src/Columns/ColumnVariant.cpp | 12 ++++++++++-- .../03228_variant_permutation_issue.reference | 4 ++++ .../03228_variant_permutation_issue.sql | 18 ++++++++++++++++++ 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03228_variant_permutation_issue.reference create mode 100644 tests/queries/0_stateless/03228_variant_permutation_issue.sql diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 28a4860b546..2fea3eca123 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1009,8 +1009,16 @@ ColumnPtr ColumnVariant::indexImpl(const PaddedPODArray & indexes, size_t new_variants.reserve(num_variants); for (size_t i = 0; i != num_variants; ++i) { - size_t nested_limit = nested_perms[i].size() == variants[i]->size() ? 0 : nested_perms[i].size(); - new_variants.emplace_back(variants[i]->permute(nested_perms[i], nested_limit)); + /// Check if no values from this variant were selected. + if (nested_perms[i].empty()) + { + new_variants.emplace_back(variants[i]->cloneEmpty()); + } + else + { + size_t nested_limit = nested_perms[i].size() == variants[i]->size() ? 0 : nested_perms[i].size(); + new_variants.emplace_back(variants[i]->permute(nested_perms[i], nested_limit)); + } } /// We cannot use new_offsets column as an offset column, because it became invalid after variants permutation. diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.reference b/tests/queries/0_stateless/03228_variant_permutation_issue.reference new file mode 100644 index 00000000000..7b18a0c59fb --- /dev/null +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.reference @@ -0,0 +1,4 @@ +2 {"foo2":"bar"} 1 +3 {"foo2":"bar"} 1 +3 {"foo2":"bar"} 1 +2 {"foo2":"baz"} 2 diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.sql b/tests/queries/0_stateless/03228_variant_permutation_issue.sql new file mode 100644 index 00000000000..3f60d42ffbd --- /dev/null +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.sql @@ -0,0 +1,18 @@ +SET allow_experimental_json_type = 1; + +DROP TABLE IF EXISTS test_new_json_type; +CREATE TABLE test_new_json_type(id UInt32, data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id; +INSERT INTO test_new_json_type format JSONEachRow +{"id":1,"data":{"foo1":"bar"},"version":1} +{"id":2,"data":{"foo2":"bar"},"version":1} +{"id":3,"data":{"foo2":"bar"},"version":1} +; + +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null; + +INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2; + +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null; + +DROP TABLE test_new_json_type; + From 725f1791f48376905da317b9891cb2860c8c8a33 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 21 Aug 2024 21:34:25 +0200 Subject: [PATCH 238/409] CI: Binary tidy timeout 3h -> 4h --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 58de25f039f..887742840a2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -163,7 +163,7 @@ class CI: tidy=True, comment="clang-tidy is used for static analysis", ), - timeout=10800, + timeout=14400, ), BuildNames.BINARY_DARWIN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( From dfe0beb53b4f0d1da50bf04e9c9e3e06f8b29ad2 Mon Sep 17 00:00:00 2001 From: Dergousov Date: Wed, 21 Aug 2024 22:46:29 +0300 Subject: [PATCH 239/409] 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 240/409] 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 d350f7bc1ab9e9454f7d84a1cc27176f3790be9b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 20:20:32 +0000 Subject: [PATCH 241/409] Improve accuracy of ICU-correcting code by 12.6% --- src/Functions/LowerUpperUTF8Impl.h | 11 ++++++----- .../0_stateless/00170_lower_upper_utf8.reference | 1 + tests/queries/0_stateless/00170_lower_upper_utf8.sql | 3 +++ 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5da085f48e5..8469bedde0c 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -43,7 +42,7 @@ struct LowerUpperUTF8Impl String output; size_t curr_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); size_t size = offsets[i] - offsets[i - 1]; @@ -57,13 +56,15 @@ struct LowerUpperUTF8Impl output.clear(); input.toUTF8String(output); - /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first + /// For valid UTF-8 input strings, ICU sometimes produces output with an extra '\0 at the end. Only the data before that /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this /// case, the behavior is also reasonable. - const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); - size_t valid_size = res_end ? res_end - output.data() + 1 : 0; + size_t valid_size = output.size(); + if (!output.empty() && output.back() == '\0') + --valid_size; res_data.resize(curr_offset + valid_size + 1); + memcpy(&res_data[curr_offset], output.data(), valid_size); res_data[curr_offset + valid_size] = 0; diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.reference b/tests/queries/0_stateless/00170_lower_upper_utf8.reference index 3c644f22b9b..b1cb9ad5b57 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.reference +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.reference @@ -26,3 +26,4 @@ 1 1 1 +2 diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.sql b/tests/queries/0_stateless/00170_lower_upper_utf8.sql index 85b6c5c6095..7c7bbac0df3 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.sql +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.sql @@ -38,3 +38,6 @@ select lowerUTF8('ır') = 'ır'; -- German language select upper('öäüß') = 'öäüß'; select lower('ÖÄÜẞ') = 'ÖÄÜẞ'; + +-- Bug 68680 +SELECT lengthUTF8(lowerUTF8('Ä\0')); From 6e5465ae5126f3281d81172e952b6811f8946f2d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 21 Aug 2024 15:47:08 +0200 Subject: [PATCH 242/409] CI: SQLLogix job fix --- docker/test/sqllogic/Dockerfile | 3 --- tests/ci/sqllogic_test.py | 10 ++++------ .../docker_scripts/sqllogic_runner.sh | 18 +++++++++--------- 3 files changed, 13 insertions(+), 18 deletions(-) rename docker/test/sqllogic/run.sh => tests/docker_scripts/sqllogic_runner.sh (87%) diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 6397526388e..0d21a2da44e 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -40,6 +40,3 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git" RUN git clone --recursive ${sqllogic_test_repo} - -COPY run.sh / -CMD ["/bin/bash", "/run.sh"] diff --git a/tests/ci/sqllogic_test.py b/tests/ci/sqllogic_test.py index 63880f07e92..7fe44c235c7 100755 --- a/tests/ci/sqllogic_test.py +++ b/tests/ci/sqllogic_test.py @@ -31,7 +31,7 @@ IMAGE_NAME = "clickhouse/sqllogic-test" def get_run_command( builds_path: Path, - repo_tests_path: Path, + repo_path: Path, result_path: Path, server_log_path: Path, image: DockerImage, @@ -39,11 +39,11 @@ def get_run_command( return ( f"docker run " f"--volume={builds_path}:/package_folder " - f"--volume={repo_tests_path}:/clickhouse-tests " + f"--volume={repo_path}:/repo " f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--cap-add=SYS_PTRACE {image}" + f"--cap-add=SYS_PTRACE {image} /repo/tests/docker_scripts/sqllogic_runner.sh" ) @@ -94,8 +94,6 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - repo_tests_path = repo_path / "tests" - packages_path = temp_path / "packages" packages_path.mkdir(parents=True, exist_ok=True) @@ -111,7 +109,7 @@ def main(): run_command = get_run_command( # run script inside docker packages_path, - repo_tests_path, + repo_path, result_path, server_log_path, docker_image, diff --git a/docker/test/sqllogic/run.sh b/tests/docker_scripts/sqllogic_runner.sh similarity index 87% rename from docker/test/sqllogic/run.sh rename to tests/docker_scripts/sqllogic_runner.sh index 32368980f9b..8b8f1e7aec7 100755 --- a/docker/test/sqllogic/run.sh +++ b/tests/docker_scripts/sqllogic_runner.sh @@ -15,10 +15,10 @@ echo "Files in current directory" ls -la ./ echo "Files in root directory" ls -la / -echo "Files in /clickhouse-tests directory" -ls -la /clickhouse-tests -echo "Files in /clickhouse-tests/sqllogic directory" -ls -la /clickhouse-tests/sqllogic +echo "Files in /repo/tests directory" +ls -la /repo/tests +echo "Files in /repo/tests/sqllogic directory" +ls -la /repo/tests/sqllogic echo "Files in /package_folder directory" ls -la /package_folder echo "Files in /test_output" @@ -45,13 +45,13 @@ function run_tests() cd /test_output - /clickhouse-tests/sqllogic/runner.py --help 2>&1 \ + /repo/tests/sqllogic/runner.py --help 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' mkdir -p /test_output/self-test - /clickhouse-tests/sqllogic/runner.py --log-file /test_output/runner-self-test.log \ + /repo/tests/sqllogic/runner.py --log-file /test_output/runner-self-test.log \ self-test \ - --self-test-dir /clickhouse-tests/sqllogic/self-test \ + --self-test-dir /repo/tests/sqllogic/self-test \ --out-dir /test_output/self-test \ 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' @@ -63,7 +63,7 @@ function run_tests() if [ -d /sqllogictest ] then mkdir -p /test_output/statements-test - /clickhouse-tests/sqllogic/runner.py \ + /repo/tests/sqllogic/runner.py \ --log-file /test_output/runner-statements-test.log \ --log-level info \ statements-test \ @@ -77,7 +77,7 @@ function run_tests() tar -zcvf statements-check.tar.gz statements-test 1>/dev/null mkdir -p /test_output/complete-test - /clickhouse-tests/sqllogic/runner.py \ + /repo/tests/sqllogic/runner.py \ --log-file /test_output/runner-complete-test.log \ --log-level info \ complete-test \ From a1f312495d4a2d18052f96aee3831fa7de2588f6 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 21 Aug 2024 22:36:00 +0200 Subject: [PATCH 243/409] CI: Integration tests timeout to 3h --- tests/ci/ci_config.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 58de25f039f..fdf863797ef 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -400,7 +400,10 @@ class CI: required_builds=[BuildNames.PACKAGE_DEBUG], pr_only=True ), JobNames.INTEGRATION_TEST_ASAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], release_only=True, num_batches=4 + required_builds=[BuildNames.PACKAGE_ASAN], + release_only=True, + num_batches=4, + timeout=10800, ), JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], From ca880ccdee16a212cebccce7090eabf4f528aa68 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Aug 2024 20:47:48 +0000 Subject: [PATCH 244/409] Fix structure comparison between 2 JSON columns --- src/Columns/ColumnObject.cpp | 4 ++-- .../03229_json_structure_comparison.reference | 3 +++ .../03229_json_structure_comparison.sql | 22 +++++++++++++++++++ 3 files changed, 27 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03229_json_structure_comparison.reference create mode 100644 tests/queries/0_stateless/03229_json_structure_comparison.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index b7194ef50e7..999c0f6088e 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -127,7 +127,7 @@ std::string ColumnObject::getName() const { WriteBufferFromOwnString ss; ss << "Object("; - ss << "max_dynamic_paths=" << max_dynamic_paths; + ss << "max_dynamic_paths=" << global_max_dynamic_paths; ss << ", max_dynamic_types=" << max_dynamic_types; std::vector sorted_typed_paths; sorted_typed_paths.reserve(typed_paths.size()); @@ -1047,7 +1047,7 @@ bool ColumnObject::structureEquals(const IColumn & rhs) const { /// 2 Object columns have equal structure if they have the same typed paths and max_dynamic_paths/max_dynamic_types. const auto * rhs_object = typeid_cast(&rhs); - if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || max_dynamic_paths != rhs_object->max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types) + if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || global_max_dynamic_paths != rhs_object->global_max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types) return false; for (const auto & [path, column] : typed_paths) diff --git a/tests/queries/0_stateless/03229_json_structure_comparison.reference b/tests/queries/0_stateless/03229_json_structure_comparison.reference new file mode 100644 index 00000000000..c816df4f5c7 --- /dev/null +++ b/tests/queries/0_stateless/03229_json_structure_comparison.reference @@ -0,0 +1,3 @@ +{"foo1":"bar"} {"foo1":"bar"} +{"foo2":"bar"} {"foo2":"bar"} +{"foo2":"bar"} {"foo2":"bar"} diff --git a/tests/queries/0_stateless/03229_json_structure_comparison.sql b/tests/queries/0_stateless/03229_json_structure_comparison.sql new file mode 100644 index 00000000000..16db469325d --- /dev/null +++ b/tests/queries/0_stateless/03229_json_structure_comparison.sql @@ -0,0 +1,22 @@ +SET allow_experimental_json_type=1; + +DROP TABLE IF EXISTS test_new_json_type; + +CREATE TABLE test_new_json_type(id UInt32, data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id; + +INSERT INTO test_new_json_type format JSONEachRow +{"id":1,"data":{"foo1":"bar"},"version":1} +{"id":2,"data":{"foo2":"bar"},"version":1} +{"id":3,"data":{"foo2":"bar"},"version":1} +; + +SELECT + a.data + , b.data +FROM + test_new_json_type a + JOIN test_new_json_type b + ON a.id = b.id; + +DROP TABLE test_new_json_type; + From bff252ea73f5141b1c85bccb69780f7e27c9a6f7 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Aug 2024 21:45:26 +0000 Subject: [PATCH 245/409] Fix test --- .../0_stateless/03228_variant_permutation_issue.reference | 2 +- tests/queries/0_stateless/03228_variant_permutation_issue.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.reference b/tests/queries/0_stateless/03228_variant_permutation_issue.reference index 7b18a0c59fb..10688253e15 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.reference +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.reference @@ -1,4 +1,4 @@ 2 {"foo2":"bar"} 1 3 {"foo2":"bar"} 1 -3 {"foo2":"bar"} 1 2 {"foo2":"baz"} 2 +3 {"foo2":"bar"} 1 diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.sql b/tests/queries/0_stateless/03228_variant_permutation_issue.sql index 3f60d42ffbd..088361d6430 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.sql +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.sql @@ -8,11 +8,11 @@ INSERT INTO test_new_json_type format JSONEachRow {"id":3,"data":{"foo2":"bar"},"version":1} ; -SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null; +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2; -SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null; +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; DROP TABLE test_new_json_type; From dbee1542b6541e6aa1634706a5ce1a681fc717c2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 23:54:19 +0200 Subject: [PATCH 246/409] It --> If --- src/Functions/LowerUpperUTF8Impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 8469bedde0c..36ee1723269 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -57,7 +57,7 @@ struct LowerUpperUTF8Impl input.toUTF8String(output); /// For valid UTF-8 input strings, ICU sometimes produces output with an extra '\0 at the end. Only the data before that - /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this + /// '\0' is valid. If the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this /// case, the behavior is also reasonable. size_t valid_size = output.size(); if (!output.empty() && output.back() == '\0') From 2f6ad1271cfbd9aa62ad2365e70314aba4da21b9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 00:27:02 +0200 Subject: [PATCH 247/409] fix tests + exception --- src/Storages/VirtualColumnUtils.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 10 +-- tests/integration/test_storage_hdfs/test.py | 9 +-- .../03203_hive_style_partitioning.reference | 2 - .../03203_hive_style_partitioning.sh | 61 +++---------------- 5 files changed, 15 insertions(+), 69 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index ca82a1ce67a..f0d276e4e56 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -162,7 +162,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return; if (storage_columns.size() == 1) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot implement partition by all columns in a file"); + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot use hive partitioning for file {}: it contains only partition columns. Disable use_hive_partitioning setting to read this file", path); auto local_type = storage_columns.get(name).type; storage_columns.remove(name); desc.addEphemeral(name, local_type, ""); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 637dbd38262..a3172329a99 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1518,14 +1518,14 @@ def test_hive_partitioning_with_one_parameter(cluster): ) query = ( - f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " + f"SELECT column2, _file, _path, column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}')" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}".format( + "Gordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path ) ] @@ -1560,7 +1560,7 @@ def test_hive_partitioning_with_all_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}');" ) - pattern = r"DB::Exception: Cannot implement partition by all columns in a file" + pattern = r"DB::Exception: Cannot use hive partitioning for file" with pytest.raises(Exception, match=pattern): azure_query(node, query, settings={"use_hive_partitioning": 1}) @@ -1572,7 +1572,7 @@ def test_hive_partitioning_without_setting(cluster): table_format = "column1 String, column2 String" values_1 = f"('Elizabeth', 'Gordon')" values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + path = "a/column1=Elizabeth/column2=Gordon/column3=Gordon/sample.csv" azure_query( node, @@ -1582,7 +1582,7 @@ def test_hive_partitioning_without_setting(cluster): ) query = ( - f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " + f"SELECT column1, column2, _file, _path, column3 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}');" ) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ad2e7084791..ea8c4efa745 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1268,11 +1268,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): ) assert r == f"Elizabeth\n" - r = node1.query( - "SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"use_hive_partitioning": 1}, - ) - assert r == f"Gordon\n" def test_hive_partitioning_with_all_parameters(started_cluster): @@ -1285,11 +1280,11 @@ def test_hive_partitioning_with_all_parameters(started_cluster): == f"Elizabeth\tGordon\n" ) - pattern = r"DB::Exception: Cannot implement partition by all columns in a file" + pattern = r"DB::Exception: Cannot use hive partitioning for file" with pytest.raises(QueryRuntimeException, match=pattern): node1.query( - f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"use_hive_partitioning": 1}, ) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index b5eaef7f51e..af52dcd9b88 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -35,8 +35,6 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 -4081 -2070 2070 b 1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 41b215578f0..4e165446c34 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -11,22 +11,10 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; @@ -37,7 +25,6 @@ SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01 $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ @@ -61,21 +48,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" @@ -92,24 +65,10 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; """ $CLICKHOUSE_CLIENT -n -q """ @@ -123,13 +82,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth'; """ From 8a89cd31a1e7770479af6eaf1b4211ef4ece1795 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Aug 2024 00:29:32 +0200 Subject: [PATCH 248/409] Fix Upgrade Check: move some settings to 24.9 section --- src/Core/SettingsChangesHistory.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index fb59577b0f0..5e831c6301c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -72,11 +72,13 @@ static std::initializer_list Date: Thu, 22 Aug 2024 00:48:29 +0200 Subject: [PATCH 249/409] fix black --- tests/integration/test_storage_hdfs/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ea8c4efa745..a75c13b9ea6 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1269,7 +1269,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" - def test_hive_partitioning_with_all_parameters(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From 0f3c7ae8c202f475fe55f33f45e9bca92155d52c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:15:16 +0200 Subject: [PATCH 250/409] 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 251/409] 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 92e153585ded4f15e1292613584ff35a55c735f3 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Wed, 21 Aug 2024 19:19:07 -0700 Subject: [PATCH 252/409] Update README.md Add latest meetups from Alexey tour. --- README.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/README.md b/README.md index 17b6dcd2ac1..5e66b9da73e 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,13 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. +The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey Milovidov: + * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 +* [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 +* [New York Meetup (Ramp)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 9d0b3e3937cca32bc8bc922876fb8e6ac53a3de9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 22 Aug 2024 11:32:59 +0800 Subject: [PATCH 253/409] change as request --- .../functions/string-replace-functions.md | 145 +++++++++--------- src/Functions/overlay.cpp | 8 +- ...new_functions_must_be_documented.reference | 2 - 3 files changed, 76 insertions(+), 79 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 55e97688b18..0cc6b0b27d5 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -8,6 +8,78 @@ sidebar_label: Replacing in Strings [General strings functions](string-functions.md) and [functions for searching in strings](string-search-functions.md) are described separately. +## overlay + +Replace part of the string `input` with another string `replace`, starting at the 1-based index `offset`. + +**Syntax** + +```sql +overlay(s, replace, offset[, length]) +``` + +**Parameters** + +- `input`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). If `offset` is negative, it is counted from the end of the `input` string. +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). `length` specifies the length of the snippet within input to be replaced. If `length` is not specified, the number of bytes removed from `input` equals the length of `replace`; otherwise `length` bytes are removed. + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +```sql +SELECT overlay('ClickHouse SQL', 'CORE', 12) AS res; +``` + +Result: + +```text +┌─res─────────────┐ +│ ClickHouse CORE │ +└─────────────────┘ +``` + +## overlayUTF8 + +Replace part of the string `input` with another string `replace`, starting at the 1-based index `offset`. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +overlayUTF8(s, replace, offset[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). If `offset` is negative, it is counted from the end of the `input` string. +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). `length` specifies the length of the snippet within input to be replaced. If `length` is not specified, the number of characters removed from `input` equals the length of `replace`; otherwise `length` characters are removed. + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +```sql +SELECT overlayUTF8('ClickHouse是一款OLAP数据库', '开源', 12, 2) AS res; +``` + +Result: + +```text +┌─res────────────────────────┐ +│ ClickHouse是开源OLAP数据库 │ +└────────────────────────────┘ +``` + ## replaceOne Replaces the first occurrence of the substring `pattern` in `haystack` by the `replacement` string. @@ -248,76 +320,3 @@ select printf('%%%s %s %d', 'Hello', 'World', 2024); │ %Hello World 2024 │ └──────────────────────────────────────────────┘ ``` - -## overlay - -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. - - -**Syntax** - -```sql -overlay(s, replace, offset[, length]) -``` - -**Parameters** - -- `s`: A string type [String](../data-types/string.md). -- `replace`: A string type [String](../data-types/string.md). -- `offset`: An integer type [Int](../data-types/int-uint.md). -- `length`: Optional. An integer type [Int](../data-types/int-uint.md). - -**Returned value** - -- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. - -**Example** - -```sql -SELECT overlay('Spark SQL', 'CORE', 7) AS res; -``` - -Result: - -```text - ┌─res────────┐ - │ Spark CORE │ - └────────────┘ -``` - -## overlayUTF8 - -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of characters removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of characters is removed. - -Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. - -**Syntax** - -```sql -overlayUTF8(s, replace, offset[, length]) -``` - -**Parameters** - -- `s`: A string type [String](../data-types/string.md). -- `replace`: A string type [String](../data-types/string.md). -- `offset`: An integer type [Int](../data-types/int-uint.md). -- `length`: Optional. An integer type [Int](../data-types/int-uint.md). - -**Returned value** - -- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. - -**Example** - -```sql -SELECT overlayUTF8('ClickHouse是一款OLAP数据库', '开源', 12, 2) AS res; -``` - -Result: - -```text -┌─res────────────────────────┐ -│ ClickHouse是开源OLAP数据库 │ -└────────────────────────────┘ -``` diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 20988c775a5..497ebb9c9cd 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -201,14 +201,14 @@ private: { if (offset > 0) { - if (static_cast(offset) > input_size + 1) [[unlikely]] + if (static_cast(offset) > input_size + 1) return input_size; else return offset - 1; } else { - if (input_size < -static_cast(offset)) [[unlikely]] + if (input_size < -static_cast(offset)) return 0; else return input_size + offset; @@ -704,14 +704,14 @@ REGISTER_FUNCTION(Overlay) { factory.registerFunction>( {.description = R"( -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. +Replace a part of a string `input` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `input` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. )", .categories{"String"}}, FunctionFactory::Case::Insensitive); factory.registerFunction>( {.description = R"( -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. +Replace a part of a string `input` with another string `replace`, starting at 1-based index `offset`. By default, the number of characters removed from `input` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of characters is removed. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. )", diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 6495b6619f9..c39f1fb1ce9 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -512,8 +512,6 @@ nullIf nullIn nullInIgnoreSet or -overlay -overlayUTF8 parseDateTime parseDateTime32BestEffort parseDateTime32BestEffortOrNull From 3ff9522b69ec7e51119f445152ffb9678a0f124f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 22 Aug 2024 12:49:10 +0800 Subject: [PATCH 254/409] change as request --- src/Functions/overlay.cpp | 165 +++++++++--------- .../0_stateless/03205_overlay.reference | 58 +++--- tests/queries/0_stateless/03205_overlay.sql | 11 +- 3 files changed, 115 insertions(+), 119 deletions(-) diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 497ebb9c9cd..df8b825eabe 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -53,139 +53,132 @@ public: if (input_rows_count == 0) return ColumnString::create(); - const size_t number_of_arguments = arguments.size(); - bool has_three_args = number_of_arguments == 3; + bool has_four_args = (arguments.size() == 4); - ColumnPtr column_offset = arguments[2].column; - ColumnPtr column_length; - if (!has_three_args) - column_length = arguments[3].column; + ColumnPtr col_input = arguments[0].column; + const auto * col_input_const = checkAndGetColumn(col_input.get()); + const auto * col_input_string = checkAndGetColumn(col_input.get()); + bool input_is_const = (col_input_const != nullptr); - const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); - const ColumnConst * column_length_const = nullptr; - if (!has_three_args) - column_length_const = checkAndGetColumn(column_length.get()); + ColumnPtr col_replace = arguments[1].column; + const auto * col_replace_const = checkAndGetColumn(col_replace.get()); + const auto * col_replace_string = checkAndGetColumn(col_replace.get()); + bool replace_is_const = (col_replace_const != nullptr); + ColumnPtr col_offset = arguments[2].column; + const ColumnConst * col_offset_const = checkAndGetColumn(col_offset.get()); bool offset_is_const = false; - bool length_is_const = false; Int64 offset = -1; - Int64 length = -1; - if (column_offset_const) + if (col_offset_const) { - offset = column_offset_const->getInt(0); + offset = col_offset_const->getInt(0); offset_is_const = true; } - if (column_length_const) + ColumnPtr col_length = has_four_args ? arguments[3].column : nullptr; + const ColumnConst * col_length_const = has_four_args ? checkAndGetColumn(col_length.get()) : nullptr; + bool length_is_const = false; + Int64 length = -1; + if (col_length_const) { - length = column_length_const->getInt(0); + length = col_length_const->getInt(0); length_is_const = true; } - auto res_col = ColumnString::create(); auto & res_data = res_col->getChars(); auto & res_offsets = res_col->getOffsets(); + res_offsets.resize_exact(input_rows_count); - - ColumnPtr column_input = arguments[0].column; - ColumnPtr column_replace = arguments[1].column; - - const auto * column_input_const = checkAndGetColumn(column_input.get()); - const auto * column_input_string = checkAndGetColumn(column_input.get()); - if (column_input_const) + if (col_input_const) { - StringRef input = column_input_const->getDataAt(0); + StringRef input = col_input_const->getDataAt(0); res_data.reserve((input.size + 1) * input_rows_count); } else { - res_data.reserve(column_input_string->getChars().size()); + res_data.reserve(col_input_string->getChars().size()); } - const auto * column_replace_const = checkAndGetColumn(column_replace.get()); - const auto * column_replace_string = checkAndGetColumn(column_replace.get()); - bool input_is_const = (column_input_const != nullptr); - bool replace_is_const = (column_replace_const != nullptr); -#define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ +#define OVERLAY_EXECUTE_CASE(HAS_FOUR_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ if (input_is_const && replace_is_const) \ - constantConstant( \ + constantConstant( \ input_rows_count, \ - column_input_const->getDataAt(0), \ - column_replace_const->getDataAt(0), \ - column_offset, \ - column_length, \ + col_input_const->getDataAt(0), \ + col_replace_const->getDataAt(0), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else if (input_is_const && !replace_is_const) \ - constantVector( \ + constantVector( \ input_rows_count, \ - column_input_const->getDataAt(0), \ - column_replace_string->getChars(), \ - column_replace_string->getOffsets(), \ - column_offset, \ - column_length, \ + col_input_const->getDataAt(0), \ + col_replace_string->getChars(), \ + col_replace_string->getOffsets(), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else if (!input_is_const && replace_is_const) \ - vectorConstant( \ + vectorConstant( \ input_rows_count, \ - column_input_string->getChars(), \ - column_input_string->getOffsets(), \ - column_replace_const->getDataAt(0), \ - column_offset, \ - column_length, \ + col_input_string->getChars(), \ + col_input_string->getOffsets(), \ + col_replace_const->getDataAt(0), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else \ - vectorVector( \ + vectorVector( \ input_rows_count, \ - column_input_string->getChars(), \ - column_input_string->getOffsets(), \ - column_replace_string->getChars(), \ - column_replace_string->getOffsets(), \ - column_offset, \ - column_length, \ + col_input_string->getChars(), \ + col_input_string->getOffsets(), \ + col_replace_string->getChars(), \ + col_replace_string->getOffsets(), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); - if (has_three_args) + if (!has_four_args) { if (offset_is_const) { - OVERLAY_EXECUTE_CASE(true, true, false) + OVERLAY_EXECUTE_CASE(false, true, false) } else { - OVERLAY_EXECUTE_CASE(true, false, false) + OVERLAY_EXECUTE_CASE(false, false, false) } } else { if (offset_is_const && length_is_const) { - OVERLAY_EXECUTE_CASE(false, true, true) + OVERLAY_EXECUTE_CASE(true, true, true) } else if (offset_is_const && !length_is_const) { - OVERLAY_EXECUTE_CASE(false, true, false) + OVERLAY_EXECUTE_CASE(true, true, false) } else if (!offset_is_const && length_is_const) { - OVERLAY_EXECUTE_CASE(false, false, true) + OVERLAY_EXECUTE_CASE(true, false, true) } else { - OVERLAY_EXECUTE_CASE(false, false, false) + OVERLAY_EXECUTE_CASE(true, false, false) } } #undef OVERLAY_EXECUTE_CASE @@ -224,7 +217,7 @@ private: return bytes; } - template + template void constantConstant( size_t rows, const StringRef & input, @@ -236,7 +229,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { constantConstant( rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -250,12 +243,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (has_three_args) + else if constexpr (!has_four_args) { valid_length = replace_size; } @@ -273,14 +266,14 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!has_three_args && !length_is_const) + if constexpr (has_four_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -332,11 +325,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void vectorConstant( size_t rows, const ColumnString::Chars & input_data, @@ -349,7 +343,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { vectorConstant( rows, input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -359,12 +353,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (has_three_args) + else if constexpr (!has_four_args) { valid_length = replace_size; } @@ -388,14 +382,14 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!has_three_args && !length_is_const) + if constexpr (has_four_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -449,11 +443,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void constantVector( size_t rows, const StringRef & input, @@ -466,7 +461,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { constantVector( rows, input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -480,7 +475,7 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; @@ -502,7 +497,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (has_three_args) + if constexpr (!has_four_args) { valid_length = replace_size; } @@ -513,7 +508,7 @@ private: } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -563,11 +558,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void vectorVector( size_t rows, const ColumnString::Chars & input_data, @@ -581,7 +577,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { vectorVector( rows, @@ -600,7 +596,7 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; @@ -629,7 +625,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (has_three_args) + if constexpr (!has_four_args) { valid_length = replace_size; } @@ -640,7 +636,7 @@ private: } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -693,6 +689,7 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference index 67a699944e0..4be3baadaea 100644 --- a/tests/queries/0_stateless/03205_overlay.reference +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -25,34 +25,34 @@ Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH Spark ANSI SQL Spark ANSI SQL和CH -Test with different offset values --12 _park SQL _park SQL和CH --11 _park SQL S_ark SQL和CH --10 _park SQL Sp_rk SQL和CH --9 _park SQL Spa_k SQL和CH --8 S_ark SQL Spar_ SQL和CH --7 Sp_rk SQL Spark_SQL和CH --6 Spa_k SQL Spark _QL和CH --5 Spar_ SQL Spark S_L和CH --4 Spark_SQL Spark SQ_和CH --3 Spark _QL Spark SQL_CH --2 Spark S_L Spark SQL和_H --1 Spark SQ_ Spark SQL和C_ -0 Spark SQL_ Spark SQL和CH_ -1 _park SQL _park SQL和CH -2 S_ark SQL S_ark SQL和CH -3 Sp_rk SQL Sp_rk SQL和CH -4 Spa_k SQL Spa_k SQL和CH -5 Spar_ SQL Spar_ SQL和CH -6 Spark_SQL Spark_SQL和CH -7 Spark _QL Spark _QL和CH -8 Spark S_L Spark S_L和CH -9 Spark SQ_ Spark SQ_和CH -10 Spark SQL_ Spark SQL_CH -11 Spark SQL_ Spark SQL和_H -12 Spark SQL_ Spark SQL和C_ -13 Spark SQL_ Spark SQL和CH_ -Test with different length values +Test with special offset values +-12 __ark SQL 之park SQL和CH +-11 __ark SQL S之ark SQL和CH +-10 __ark SQL Sp之rk SQL和CH +-9 __ark SQL Spa之k SQL和CH +-8 S__rk SQL Spar之 SQL和CH +-7 Sp__k SQL Spark之SQL和CH +-6 Spa__ SQL Spark 之QL和CH +-5 Spar__SQL Spark S之L和CH +-4 Spark__QL Spark SQ之和CH +-3 Spark __L Spark SQL之CH +-2 Spark S__ Spark SQL和之H +-1 Spark SQ__ Spark SQL和C之 +0 Spark SQL__ Spark SQL和CH之 +1 __ark SQL 之park SQL和CH +2 S__rk SQL S之ark SQL和CH +3 Sp__k SQL Sp之rk SQL和CH +4 Spa__ SQL Spa之k SQL和CH +5 Spar__SQL Spar之 SQL和CH +6 Spark__QL Spark之SQL和CH +7 Spark __L Spark 之QL和CH +8 Spark S__ Spark S之L和CH +9 Spark SQ__ Spark SQ之和CH +10 Spark SQL__ Spark SQL之CH +11 Spark SQL__ Spark SQL和之H +12 Spark SQL__ Spark SQL和C之 +13 Spark SQL__ Spark SQL和CH之 +Test with special length values -1 Spark ANSI Spark ANSI H 0 Spark ANSI SQL Spark ANSI SQL和CH 1 Spark ANSI QL Spark ANSI QL和CH @@ -61,7 +61,7 @@ Test with different length values 4 Spark ANSI Spark ANSI CH 5 Spark ANSI Spark ANSI H 6 Spark ANSI Spark ANSI -Test with different input and replace values +Test with special input and replace values _ _ Spark SQL Spark SQL和CH ANSI ANSI diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql index 4d0b5ecbe03..765b29f93ec 100644 --- a/tests/queries/0_stateless/03205_overlay.sql +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -1,5 +1,4 @@ SELECT 'Negative test of overlay'; -SELECT overlay('hello', 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay('hello', 'world'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay('hello', 'world', 2, 3, 'extra'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay(123, 'world', 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } @@ -35,13 +34,13 @@ SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0) SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8('Spark SQL和CH', materialize('ANSI '), materialize(7), materialize(0)); SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQL和CH'), materialize('ANSI '), materialize(7), materialize(0)); -SELECT 'Test with different offset values'; -WITH number - 12 as offset SELECT offset, overlay('Spark SQL', '_', offset), overlayUTF8('Spark SQL和CH', '_', offset) from numbers(26); +SELECT 'Test with special offset values'; +WITH number - 12 AS offset SELECT offset, overlay('Spark SQL', '__', offset), overlayUTF8('Spark SQL和CH', '之', offset) FROM numbers(26); -SELECT 'Test with different length values'; -WITH number - 1 as length SELECT length, overlay('Spark SQL', 'ANSI ', 7, length), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, length) from numbers(8); +SELECT 'Test with special length values'; +WITH number - 1 AS length SELECT length, overlay('Spark SQL', 'ANSI ', 7, length), overlayUTF8('Spark SQL和CH', 'ANSI ', 7, length) FROM numbers(8); -SELECT 'Test with different input and replace values'; +SELECT 'Test with special input and replace values'; SELECT overlay('', '_', 6), overlayUTF8('', '_', 6); SELECT overlay('Spark SQL', '', 6), overlayUTF8('Spark SQL和CH', '', 6); SELECT overlay('', 'ANSI ', 7, 0), overlayUTF8('', 'ANSI ', 7, 0); From be4439e3ec0a1491f4e333ac848844fd930a6e5b Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 22 Aug 2024 10:30:48 +0300 Subject: [PATCH 255/409] Update install.md Added correct commands for russian vwersion of the installation from deb packets --- docs/ru/getting-started/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index aee445da843..4a0ec258c64 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -25,10 +25,10 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su Яндекс рекомендует использовать официальные скомпилированные `deb`-пакеты для Debian или Ubuntu. Для установки пакетов выполните: ``` bash -sudo apt-get install -y apt-transport-https ca-certificates dirmngr -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 +sudo apt-get install -y apt-transport-https ca-certificates curl gnupg +curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | sudo gpg --dearmor -o /usr/share/keyrings/clickhouse-keyring.gpg -echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ +echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update From 6466f374e0372b22a23d1193e534bd6c94f87b94 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 22 Aug 2024 11:29:33 +0200 Subject: [PATCH 256/409] Update geohash.md --- .../en/sql-reference/functions/geo/geohash.md | 31 ++++++++++++++----- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index b6ac7a74092..c4f41fc53da 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -6,7 +6,7 @@ title: "Functions for Working with Geohash" ## Geohash -[Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer is the geohash string, the more precise is the geographic location. +[Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer the geohash string is, the more precise the geographic location will be. If you need to manually convert geographic coordinates to geohash strings, you can use [geohash.org](http://geohash.org/). @@ -14,26 +14,37 @@ If you need to manually convert geographic coordinates to geohash strings, you c Encodes latitude and longitude as a [geohash](#geohash)-string. +**Syntax** + ``` sql geohashEncode(longitude, latitude, [precision]) ``` **Input values** -- longitude - longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]` -- latitude - latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]` -- precision - Optional, length of the resulting encoded string, defaults to `12`. Integer in range `[1, 12]`. Any value less than `1` or greater than `12` is silently converted to `12`. +- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data_types/float.md). +- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data_types/float.md). +- `precision` (optional) — Length of the resulting encoded string. Defaults to `12`. Integer in the range `[1, 12]`. [Int8](../../data-types/int-uint.md). + +:::note +- All coordinate parameters must be of the same type: either `Float32` or `Float64`. +- For the `precision` parameter, any value less than `1` or greater than `12` is silently converted to `12`. +::: **Returned values** -- alphanumeric `String` of encoded coordinate (modified version of the base32-encoding alphabet is used). +- Alphanumeric string of the encoded coordinate (modified version of the base32-encoding alphabet is used). [String](../../data-types/string.md). **Example** +Query: + ``` sql SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; ``` +Result: + ``` text ┌─res──────────┐ │ ezs42d000000 │ @@ -44,13 +55,19 @@ SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; Decodes any [geohash](#geohash)-encoded string into longitude and latitude. +**Syntax** + +```sql +geohashDecode(hash_str) +``` + **Input values** -- encoded string - geohash-encoded string. +- `hash_str` — Geohash-encoded string. **Returned values** -- (longitude, latitude) - 2-tuple of `Float64` values of longitude and latitude. +- Tuple `(longitude, latitude)` of `Float64` values of longitude and latitude. [Tuple](../../data-types/tuple.md)([Float64](../../data-types/float.md)) **Example** From 526be4759d40efad45a871c3ff150929fcca76be Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Thu, 22 Aug 2024 10:34:05 +0000 Subject: [PATCH 257/409] SQL as a set language, doesn't have order in the output, so add ORDER BY or min call for it --- .../integration/test_truncate_database/test_replicated.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_truncate_database/test_replicated.py b/tests/integration/test_truncate_database/test_replicated.py index 59830d44378..d55a0dc7557 100644 --- a/tests/integration/test_truncate_database/test_replicated.py +++ b/tests/integration/test_truncate_database/test_replicated.py @@ -1,5 +1,3 @@ -import time - import pytest from helpers.cluster import ClickHouseCluster @@ -9,7 +7,7 @@ def fill_nodes(nodes, shard): node.query( """ CREATE DATABASE test; - + CREATE TABLE test.test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date); """.format( @@ -50,7 +48,8 @@ def test_truncate_database_replicated(start_cluster): node1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) - assert node2.query("SELECT id FROM test.test_table LIMIT 1") == "0\n" + assert node2.query("SELECT min(id) FROM test.test_table") == "0\n" + assert node2.query("SELECT id FROM test.test_table ORDER BY id LIMIT 1") == "0\n" assert node3.query("SHOW DATABASES LIKE 'test'") == "test\n" node3.query("TRUNCATE DATABASE test ON CLUSTER test_cluster SYNC") assert node2.query("SHOW TABLES FROM test") == "" From 95f45d2eaf39a9e8a6373c75749ec57f727be700 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:20:04 +0200 Subject: [PATCH 258/409] try to fix tests --- .../test_storage_azure_blob_storage/test.py | 14 +++++------ tests/integration/test_storage_hdfs/test.py | 25 +++---------------- .../03203_hive_style_partitioning.reference | 20 +++++++-------- .../03203_hive_style_partitioning.sh | 14 +++-------- 4 files changed, 23 insertions(+), 50 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index a3172329a99..c1f518e45ce 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1513,14 +1513,14 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column2, _file, _path, column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}')" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}')" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} @@ -1533,7 +1533,7 @@ def test_hive_partitioning_with_one_parameter(cluster): query = ( f"SELECT column2 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} @@ -1551,14 +1551,14 @@ def test_hive_partitioning_with_all_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) pattern = r"DB::Exception: Cannot use hive partitioning for file" @@ -1577,14 +1577,14 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column1, column2, _file, _path, column3 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) pattern = re.compile( r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index a75c13b9ea6..31cc8609eb4 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1259,35 +1259,16 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): def test_hive_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" + hdfs_api.write_data(f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/file_1") == f"column0,column1\nElizabeth,Gordon\n" r = node1.query( - "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/file_1', 'CSVWithNames')", settings={"use_hive_partitioning": 1}, ) assert r == f"Elizabeth\n" -def test_hive_partitioning_with_all_parameters(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - - pattern = r"DB::Exception: Cannot use hive partitioning for file" - - with pytest.raises(QueryRuntimeException, match=pattern): - node1.query( - f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"use_hive_partitioning": 1}, - ) - - def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index af52dcd9b88..acdadc2510b 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -1,5 +1,5 @@ TESTING THE FILE HIVE PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -19,8 +19,7 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -35,12 +34,13 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 +4081 +2070 2070 b 1 -1 TESTING THE URL PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -60,10 +60,9 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth 1 TESTING THE S3 PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -83,8 +82,7 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -96,7 +94,7 @@ Delgado Elizabeth Cross Elizabeth OK TESTING THE S3CLUSTER PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -106,7 +104,7 @@ Gibson Elizabeth Greer Elizabeth Delgado Elizabeth Cross Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 4e165446c34..b3d196924af 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -14,7 +14,7 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; @@ -29,16 +29,10 @@ SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.c SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ -$CLICKHOUSE_LOCAL -n -q """ -set use_hive_partitioning = 1; - -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; -""" 2>&1 | grep -c "INCORRECT_DATA" - $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -68,7 +62,7 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; """ $CLICKHOUSE_CLIENT -n -q """ @@ -84,5 +78,5 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; """ From 62054cae666244fd072a56f70a6df73e68249cb0 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:49:16 +0200 Subject: [PATCH 259/409] Update geohash.md --- docs/en/sql-reference/functions/geo/geohash.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index c4f41fc53da..ce2e3c43b3e 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -22,8 +22,8 @@ geohashEncode(longitude, latitude, [precision]) **Input values** -- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data_types/float.md). -- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data_types/float.md). +- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data-types/float.md). +- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data-types/float.md). - `precision` (optional) — Length of the resulting encoded string. Defaults to `12`. Integer in the range `[1, 12]`. [Int8](../../data-types/int-uint.md). :::note From 84467077b886cd48c9cd33c69c1935b3f7863dd7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 22 Aug 2024 13:45:13 +0200 Subject: [PATCH 260/409] Fix test for role expiration in RoleCache. --- tests/integration/test_role/test.py | 81 +++++++++-------------------- 1 file changed, 26 insertions(+), 55 deletions(-) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index b3b18dc8271..9d15f0f81db 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -418,72 +418,43 @@ def test_function_current_roles(): ) -def test_role_expiration(): - instance.query("CREATE USER ure") +@pytest.mark.parametrize("with_extra_role", [False, True]) +def test_role_expiration(with_extra_role): instance.query("CREATE ROLE rre") - instance.query("GRANT rre TO ure") + instance.query("CREATE USER ure DEFAULT ROLE rre") - instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") - instance.query("INSERT INTO tre VALUES (0)") + instance.query("CREATE TABLE table1 (id Int) Engine=Log") + instance.query("CREATE TABLE table2 (id Int) Engine=Log") + instance.query("INSERT INTO table1 VALUES (1)") + instance.query("INSERT INTO table2 VALUES (2)") + instance.query("GRANT SELECT ON table1 TO rre") + + assert instance.query("SELECT * FROM table1", user="ure") == "1\n" assert "Not enough privileges" in instance.query_and_get_error( - "SELECT * FROM tre", user="ure" + "SELECT * FROM table2", user="ure" ) - instance.query("GRANT SELECT ON tre TO rre") - - assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test # so we wait >2 seconds until the role is expired time.sleep(5) - instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") - instance.query("INSERT INTO tre1 VALUES (0)") - instance.query("GRANT SELECT ON tre1 TO rre") + if with_extra_role: + # Expiration of role "rre" from the role cache can be caused by another role being used. + instance.query("CREATE ROLE extra_role") + instance.query("CREATE USER extra_user DEFAULT ROLE extra_role") + instance.query("GRANT SELECT ON table1 TO extra_role") + assert instance.query("SELECT * FROM table1", user="extra_user") == "1\n" - assert instance.query("SELECT * from tre1", user="ure") == "0\n" + instance.query("GRANT SELECT ON table2 TO rre") + assert instance.query("SELECT * FROM table1", user="ure") == "1\n" + assert instance.query("SELECT * FROM table2", user="ure") == "2\n" - instance.query("DROP USER ure") instance.query("DROP ROLE rre") - instance.query("DROP TABLE tre") - instance.query("DROP TABLE tre1") - - -def test_two_roles_expiration(): - instance.query("CREATE USER ure") - instance.query("CREATE ROLE rre") - instance.query("GRANT rre TO ure") - - instance.query("CREATE ROLE rre_second") - - instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") - instance.query("INSERT INTO tre VALUES (0)") - - assert "Not enough privileges" in instance.query_and_get_error( - "SELECT * FROM tre", user="ure" - ) - - instance.query("GRANT SELECT ON tre TO rre") - - assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - - # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test - # so we wait >2 seconds until the roles are expired - time.sleep(5) - - instance.query( - "GRANT SELECT ON tre1 TO rre_second" - ) # we expect that both rre and rre_second are gone from cache upon this operation - - instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") - instance.query("INSERT INTO tre1 VALUES (0)") - instance.query("GRANT SELECT ON tre1 TO rre") - - assert instance.query("SELECT * from tre1", user="ure") == "0\n" - instance.query("DROP USER ure") - instance.query("DROP ROLE rre") - instance.query("DROP ROLE rre_second") - instance.query("DROP TABLE tre") - instance.query("DROP TABLE tre1") + instance.query("DROP TABLE table1") + instance.query("DROP TABLE table2") + + if with_extra_role: + instance.query("DROP ROLE extra_role") + instance.query("DROP USER extra_user") From 664e9b3db9d47e45c642ad21e3a5273ab423199a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Aug 2024 13:30:41 +0200 Subject: [PATCH 261/409] Add one more test. --- tests/integration/test_role/test.py | 173 ++++++++++++++++++++++++++++ 1 file changed, 173 insertions(+) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 9d15f0f81db..225cab975ff 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -1,5 +1,6 @@ import time import pytest +import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -458,3 +459,175 @@ def test_role_expiration(with_extra_role): if with_extra_role: instance.query("DROP ROLE extra_role") instance.query("DROP USER extra_user") + + +def test_roles_cache(): + # This test takes 20 seconds. + test_time = 20 + + # Three users A, B, C. + users = ["A", "B", "C"] + instance.query("CREATE USER " + ", ".join(users)) + + # Table "tbl" has 10 columns. Each of the users has access to a different set of columns. + num_columns = 10 + columns = [f"x{i}" for i in range(1, num_columns + 1)] + columns_with_types = [column + " Int64" for column in columns] + columns_with_types_comma_separated = ", ".join(columns_with_types) + values = list(range(1, num_columns + 1)) + values_comma_separated = ", ".join([str(value) for value in values]) + instance.query( + f"CREATE TABLE tbl ({columns_with_types_comma_separated}) ENGINE=MergeTree ORDER BY tuple()" + ) + instance.query(f"INSERT INTO tbl VALUES ({values_comma_separated})") + columns_to_values = dict([(f"x{i}", i) for i in range(1, num_columns + 1)]) + + # In this test we create and modify roles multiple times along with updating the following variables. + # Then we check that each of the users has access to the expected set of columns. + roles = [] + users_to_roles = dict([(user, []) for user in users]) + roles_to_columns = {} + + # Checks that each of the users can access the expected set of columns and can't access other columns. + def check(): + for user in random.sample(users, len(users)): + expected_roles = users_to_roles[user] + expected_columns = list( + set(sum([roles_to_columns[role] for role in expected_roles], [])) + ) + expected_result = sorted( + [columns_to_values[column] for column in expected_columns] + ) + query = " UNION ALL ".join( + [ + f"SELECT * FROM viewIfPermitted(SELECT {column} AS c FROM tbl ELSE null('c Int64'))" + for column in columns + ] + ) + result = instance.query(query, user=user).splitlines() + result = sorted([int(value) for value in result]) + ok = result == expected_result + if not ok: + print(f"Show grants for {user}:") + print( + instance.query( + "SHOW GRANTS FOR " + ", ".join([user] + expected_roles) + ) + ) + print(f"Expected result: {expected_result}") + print(f"Got unexpected result: {result}") + assert ok + + # Grants one of our roles a permission to access one of the columns. + def grant_column(): + columns_used_in_roles = sum(roles_to_columns.values(), []) + columns_to_choose = [ + column for column in columns if column not in columns_used_in_roles + ] + if not columns_to_choose or not roles: + return False + column = random.choice(columns_to_choose) + role = random.choice(roles) + instance.query(f"GRANT SELECT({column}) ON tbl TO {role}") + roles_to_columns[role].append(column) + return True + + # Revokes a permission to access one of the granted column from all our roles. + def revoke_column(): + columns_used_in_roles = sum(roles_to_columns.values(), []) + columns_to_choose = list(set(columns_used_in_roles)) + if not columns_to_choose or not roles: + return False + column = random.choice(columns_to_choose) + roles_str = ", ".join(roles) + instance.query(f"REVOKE SELECT({column}) ON tbl FROM {roles_str}") + for role in roles_to_columns: + if column in roles_to_columns[role]: + roles_to_columns[role].remove(column) + return True + + # Creates a role and grants it to one of the users. + def create_role(): + for role in ["R1", "R2", "R3"]: + if role not in roles: + instance.query(f"CREATE ROLE {role}") + roles.append(role) + if role not in roles_to_columns: + roles_to_columns[role] = [] + if "R1" not in users_to_roles["A"]: + instance.query("GRANT R1 TO A") + users_to_roles["A"].append("R1") + elif "R2" not in users_to_roles["B"]: + instance.query("GRANT R2 TO B") + users_to_roles["B"].append("R2") + elif "R3" not in users_to_roles["B"]: + instance.query("GRANT R3 TO R2") + users_to_roles["B"].append("R3") + elif "R3" not in users_to_roles["C"]: + instance.query("GRANT R3 TO C") + users_to_roles["C"].append("R3") + else: + return False + return True + + # Drops one of our roles. + def drop_role(): + if not roles: + return False + role = random.choice(roles) + instance.query(f"DROP ROLE {role}") + roles.remove(role) + for u in users_to_roles: + if role in users_to_roles[u]: + users_to_roles[u].remove(role) + del roles_to_columns[role] + if (role == "R2") and ("R3" in users_to_roles["B"]): + users_to_roles["B"].remove("R3") + return True + + # Modifies some grants or roles randomly. + def modify(): + while True: + rnd = random.random() + if rnd < 0.4: + if grant_column(): + break + elif rnd < 0.5: + if revoke_column(): + break + elif rnd < 0.9: + if create_role(): + break + else: + if drop_role(): + break + + def maybe_modify(): + if random.random() < 0.9: + modify() + modify() + + # Sleeping is necessary in this test because the role cache in ClickHouse has expiration timeout. + def maybe_sleep(): + if random.random() < 0.1: + # "role_cache_expiration_time_seconds" is set to 2 seconds in the test configuration. + # We need a sleep longer than that in this test sometimes. + seconds = random.random() * 5 + print(f"Sleeping {seconds} seconds") + time.sleep(seconds) + + # Main part of the test. + start_time = time.time() + end_time = start_time + test_time + + while time.time() < end_time: + check() + maybe_sleep() + maybe_modify() + maybe_sleep() + + check() + + instance.query("DROP USER " + ", ".join(users)) + instance.query("DROP ROLE " + ", ".join(roles)) + instance.query("DROP TABLE tbl") From 7ef5c366e873c4fd99f257eefbb3a350848e308c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Aug 2024 13:33:50 +0200 Subject: [PATCH 262/409] Fix expiration in RoleCache. --- src/Access/RoleCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 2d94df2eea5..cc1f1520b67 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -120,7 +120,7 @@ void RoleCache::collectEnabledRoles(EnabledRoles & enabled_roles, SubscriptionsO SubscriptionsOnRoles new_subscriptions_on_roles; new_subscriptions_on_roles.reserve(subscriptions_on_roles.size()); - auto get_role_function = [this, &subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, subscriptions_on_roles); }; + auto get_role_function = [this, &new_subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, new_subscriptions_on_roles); }; for (const auto & current_role : enabled_roles.params.current_roles) collectRoles(*new_info, skip_ids, get_role_function, current_role, true, false); From 34eca80837837b1856d2d00bb86290abe41ea20c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 22 Aug 2024 12:34:56 +0000 Subject: [PATCH 263/409] Speedup test 02150_index_hypothesis_race_long --- .../0_stateless/02150_index_hypothesis_race_long.sh | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index 5c432350768..92b78a6d27a 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -11,13 +11,21 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDE $CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)" -for _ in {0..30}; do +run_query() { output=`$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_index_hypothesis WHERE a = b"` if [[ $output != "0" ]]; then echo "output: $output, expected: 0" exit 1 fi -done +} + +export -f run_query +parallel -j 8 run_query ::: {0..30} + +if [ $? -ne 0 ]; then + echo FAILED + exit 1 +fi echo OK From 54dd3afd49df9c92cd3621a5cec4c7464c341a71 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 14:52:17 +0200 Subject: [PATCH 264/409] Turn off fault injection for insert in 01396_inactive_replica_cleanup_nodes_zookeeper --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index bff85b3e29f..9ea15071856 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -23,11 +23,10 @@ $CLICKHOUSE_CLIENT -n --query " DETACH TABLE r2; " -$CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" +# insert_keeper_fault_injection_probability=0 -- can slowdown insert a lot (produce a lot of parts) +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" -# Now wait for cleanup thread - for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; From 7a740819b9551a291827b9d37b8b724612587a20 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 14:53:15 +0200 Subject: [PATCH 265/409] Accidentally deleted comment --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 9ea15071856..80e9253af2c 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT -n --query " $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" +# Now wait for cleanup thread for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; From b3f084459f60b1e31c32736573af0810dee99230 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 14:53:53 +0200 Subject: [PATCH 266/409] fix black --- tests/integration/test_storage_hdfs/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 31cc8609eb4..b18940b7290 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1259,8 +1259,13 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): def test_hive_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/file_1") == f"column0,column1\nElizabeth,Gordon\n" + hdfs_api.write_data( + f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/file_1") + == f"column0,column1\nElizabeth,Gordon\n" + ) r = node1.query( "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/file_1', 'CSVWithNames')", @@ -1269,6 +1274,7 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" + def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From 8d14d8523098a42cd778ef50a9b066508da8919c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 15:24:33 +0200 Subject: [PATCH 267/409] fix black --- tests/integration/test_storage_hdfs/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index b18940b7290..7a92e8adb0d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1274,7 +1274,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" - def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From add4718634317304f652579a9f201c3b81c96a7d Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Thu, 22 Aug 2024 06:37:27 -0700 Subject: [PATCH 268/409] Update README.md - Meetups update Fixed one meetup location; Added more meetups --- README.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5e66b9da73e..c9474ef0fc0 100644 --- a/README.md +++ b/README.md @@ -45,9 +45,17 @@ The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 * [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 -* [New York Meetup (Ramp)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 * [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 +Other upcoming meetups +* [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 +* [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 +* [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 +* [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 +* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 + ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" * **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now! From 91e65feaaedd4806875aed3d4be4f07edeefdb71 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 13:42:30 +0000 Subject: [PATCH 269/409] fix virtual columns in Merge engine --- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMerge.cpp | 14 +++++++------- .../02890_describe_table_options.reference | 8 ++++++++ 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c4668159759..0b80858800b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,7 +290,7 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); - /// Add virtual columns from table of storage Merges. + /// Add virtual columns from table with Merge engine. desc.addEphemeral("_database", std::make_shared(std::make_shared()), "The name of database which the row comes from"); desc.addEphemeral("_table", std::make_shared(std::make_shared()), "The name of table which the row comes from"); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e88844e2d31..0827321e296 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -642,10 +642,6 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } } - else - { - - } auto child = createPlanForTable( nested_storage_snaphsot, @@ -657,6 +653,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ row_policy_data_opt, modified_context, current_streams); + child.plan.addInterpreterContext(modified_context); if (child.plan.isInitialized()) @@ -914,12 +911,14 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo modified_query_info.table_expression = replacement_table_expression; modified_query_info.planner_context->getOrCreateTableExpressionData(replacement_table_expression); - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); - if (storage_snapshot_->storage.supportsSubcolumns()) - get_column_options.withSubcolumns(); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All) + .withExtendedObjects() + .withSubcolumns(storage_snapshot_->storage.supportsSubcolumns()); std::unordered_map column_name_to_node; + /// Consider only non-virtual columns of storage while checking for _table and _database columns. + /// I.e. always override virtual columns with these names from underlying table (if any). if (!storage_snapshot_->tryGetColumn(get_column_options, "_table")) { auto table_name_node = std::make_shared(current_storage_id.table_name); @@ -946,6 +945,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo column_name_to_node.emplace("_database", function_node); } + get_column_options.withVirtuals(); auto storage_columns = storage_snapshot_->metadata->getColumns(); bool with_aliases = /* common_processed_stage == QueryProcessingStage::FetchColumns && */ !storage_columns.getAliases().empty(); diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 9181cb27cb0..b77ef4a0fdf 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -54,6 +54,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 1 _shard_num UInt32 Deprecated. Use function shardNum instead 1 +_database LowCardinality(String) The name of database which the row comes from 1 +_table LowCardinality(String) The name of table which the row comes from 1 SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 index column 0 0 @@ -87,6 +89,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 0 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 0 1 _shard_num UInt32 Deprecated. Use function shardNum instead 0 1 +_database LowCardinality(String) The name of database which the row comes from 0 1 +_table LowCardinality(String) The name of table which the row comes from 0 1 arr.size0 UInt64 1 0 t.a String ZSTD(1) 1 0 t.b UInt64 ZSTD(1) 1 0 @@ -144,6 +148,8 @@ _row_exists UInt8 1 _block_number UInt64 1 _block_offset UInt64 1 _shard_num UInt32 1 +_database LowCardinality(String) 1 +_table LowCardinality(String) 1 SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 0 0 @@ -177,6 +183,8 @@ _row_exists UInt8 0 1 _block_number UInt64 0 1 _block_offset UInt64 0 1 _shard_num UInt32 0 1 +_database LowCardinality(String) 0 1 +_table LowCardinality(String) 0 1 arr.size0 UInt64 1 0 t.a String 1 0 t.b UInt64 1 0 From ce33943b430a9ad512f4942083889dea4decb778 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Aug 2024 15:50:59 +0200 Subject: [PATCH 270/409] Fix flaky check --- tests/docker_scripts/stateless_runner.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker_scripts/stateless_runner.sh b/tests/docker_scripts/stateless_runner.sh index 40a63f74a6b..d8921a04458 100755 --- a/tests/docker_scripts/stateless_runner.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -339,7 +339,7 @@ export -f run_tests if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. - NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests \ + NUM_TRIES=1 USE_DATABASE_ORDINARY=1 run_tests \ | sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||: fi From a9e793532ae308767da3bc4da74d9631cd85eb90 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 16:34:14 +0200 Subject: [PATCH 271/409] fix shutdown for PeriodicLog --- src/Interpreters/PeriodicLog.cpp | 3 ++- src/Interpreters/PeriodicLog.h | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 9d2891e11eb..15970ca5b81 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Functions/DateTimeTransforms.h" namespace DB { @@ -27,7 +28,7 @@ template void PeriodicLog::shutdown() { stopCollect(); - this->stopFlushThread(); + Base::shutdown(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index 08c3f7eb23f..ceac8088d40 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -17,6 +17,7 @@ template class PeriodicLog : public SystemLog { using SystemLog::SystemLog; + using Base = SystemLog; public: using TimePoint = std::chrono::system_clock::time_point; @@ -24,12 +25,12 @@ public: /// Launches a background thread to collect metrics with interval void startCollect(size_t collect_interval_milliseconds_); - /// Stop background thread - void stopCollect(); - void shutdown() final; protected: + /// Stop background thread + void stopCollect(); + virtual void stepFunction(TimePoint current_time) = 0; private: From 5340ac5fbc7fba75d6a743d345c0f79dc466df0b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 14:39:19 +0000 Subject: [PATCH 272/409] Update version_date.tsv and changelogs after v24.5.5.41-stable --- docs/changelogs/v24.5.5.41-stable.md | 71 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 + 2 files changed, 73 insertions(+) create mode 100644 docs/changelogs/v24.5.5.41-stable.md diff --git a/docs/changelogs/v24.5.5.41-stable.md b/docs/changelogs/v24.5.5.41-stable.md new file mode 100644 index 00000000000..8ba160e31d7 --- /dev/null +++ b/docs/changelogs/v24.5.5.41-stable.md @@ -0,0 +1,71 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.5.41-stable (441d4a6ebe3) FIXME as compared to v24.5.4.49-stable (63b760955a0) + +#### Improvement +* Backported in [#66768](https://github.com/ClickHouse/ClickHouse/issues/66768): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65350](https://github.com/ClickHouse/ClickHouse/issues/65350): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65621](https://github.com/ClickHouse/ClickHouse/issues/65621): Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67902](https://github.com/ClickHouse/ClickHouse/issues/67902): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66884](https://github.com/ClickHouse/ClickHouse/issues/66884): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65933](https://github.com/ClickHouse/ClickHouse/issues/65933): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#66301](https://github.com/ClickHouse/ClickHouse/issues/66301): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66328](https://github.com/ClickHouse/ClickHouse/issues/66328): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68252](https://github.com/ClickHouse/ClickHouse/issues/68252): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#66155](https://github.com/ClickHouse/ClickHouse/issues/66155): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66454](https://github.com/ClickHouse/ClickHouse/issues/66454): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66226](https://github.com/ClickHouse/ClickHouse/issues/66226): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66680](https://github.com/ClickHouse/ClickHouse/issues/66680): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66604](https://github.com/ClickHouse/ClickHouse/issues/66604): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66360](https://github.com/ClickHouse/ClickHouse/issues/66360): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68064](https://github.com/ClickHouse/ClickHouse/issues/68064): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68158](https://github.com/ClickHouse/ClickHouse/issues/68158): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#66972](https://github.com/ClickHouse/ClickHouse/issues/66972): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66691](https://github.com/ClickHouse/ClickHouse/issues/66691): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#66969](https://github.com/ClickHouse/ClickHouse/issues/66969): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66720](https://github.com/ClickHouse/ClickHouse/issues/66720): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66951](https://github.com/ClickHouse/ClickHouse/issues/66951): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66757](https://github.com/ClickHouse/ClickHouse/issues/66757): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66948](https://github.com/ClickHouse/ClickHouse/issues/66948): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68115](https://github.com/ClickHouse/ClickHouse/issues/68115): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67633](https://github.com/ClickHouse/ClickHouse/issues/67633): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67481](https://github.com/ClickHouse/ClickHouse/issues/67481): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67814](https://github.com/ClickHouse/ClickHouse/issues/67814): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67197](https://github.com/ClickHouse/ClickHouse/issues/67197): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67379](https://github.com/ClickHouse/ClickHouse/issues/67379): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67501](https://github.com/ClickHouse/ClickHouse/issues/67501): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67886](https://github.com/ClickHouse/ClickHouse/issues/67886): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67576](https://github.com/ClickHouse/ClickHouse/issues/67576): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67850](https://github.com/ClickHouse/ClickHouse/issues/67850): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68272](https://github.com/ClickHouse/ClickHouse/issues/68272): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67807](https://github.com/ClickHouse/ClickHouse/issues/67807): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67836](https://github.com/ClickHouse/ClickHouse/issues/67836): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67991](https://github.com/ClickHouse/ClickHouse/issues/67991): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68207](https://github.com/ClickHouse/ClickHouse/issues/68207): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68091](https://github.com/ClickHouse/ClickHouse/issues/68091): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68122](https://github.com/ClickHouse/ClickHouse/issues/68122): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68171](https://github.com/ClickHouse/ClickHouse/issues/68171): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68337](https://github.com/ClickHouse/ClickHouse/issues/68337): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68667](https://github.com/ClickHouse/ClickHouse/issues/68667): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66387](https://github.com/ClickHouse/ClickHouse/issues/66387): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66426](https://github.com/ClickHouse/ClickHouse/issues/66426): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66544](https://github.com/ClickHouse/ClickHouse/issues/66544): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66859](https://github.com/ClickHouse/ClickHouse/issues/66859): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66875](https://github.com/ClickHouse/ClickHouse/issues/66875): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67059](https://github.com/ClickHouse/ClickHouse/issues/67059): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66945](https://github.com/ClickHouse/ClickHouse/issues/66945): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67252](https://github.com/ClickHouse/ClickHouse/issues/67252): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67412](https://github.com/ClickHouse/ClickHouse/issues/67412): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). +* Update version after release. [#67862](https://github.com/ClickHouse/ClickHouse/pull/67862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68077](https://github.com/ClickHouse/ClickHouse/issues/68077): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..9063d3ef971 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +15,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 1ea0163dfe6b3278d8a5e8d86c31b3d63d7a3780 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:42:14 +0200 Subject: [PATCH 273/409] Fix issue with maps with arrays as keys --- src/Functions/FunctionsHashing.h | 4 ++-- tests/queries/0_stateless/02534_keyed_siphash.reference | 7 ++++++- tests/queries/0_stateless/02534_keyed_siphash.sql | 5 ++++- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 0cf4246fd66..3da0b2cd9be 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -93,9 +93,9 @@ namespace impl if (is_const) i = 0; assert(key0->size() == key1->size()); - if (offsets != nullptr) + if (offsets != nullptr && i > 0) { - const auto * const begin = offsets->begin(); + const auto * const begin = std::upper_bound(offsets->begin(), offsets->end(), i - 1); const auto * upper = std::upper_bound(begin, offsets->end(), i); if (upper != offsets->end()) i = upper - begin; diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 31c0cae8981..8b147025a05 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -244,5 +244,10 @@ Test emtpy arrays and maps 0AD04BFD000000000000000000000000 4761183170873013810 0AD04BFD000000000000000000000000 +Test maps with arrays as keys 16734549324845627102 -D675BB3D687973A238AB891DD99C7047 +1D03941D808D04810D2363A6C107D622 +16734549324845627102 +16734549324845627102 +1D03941D808D04810D2363A6C107D622 +1D03941D808D04810D2363A6C107D622 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index b499d8ef02b..ba3c4a9156d 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -351,5 +351,8 @@ SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); +SELECT 'Test maps with arrays as keys'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3)); -SELECT hex(sipHash128Keyed((0::UInt64, 0::UInt64), map([0], 1, [2], 3))); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3))); +SELECT sipHash64Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3)) FROM numbers(2); +SELECT hex(sipHash128Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3))) FROM numbers(2); From a93d1919804d1c8dc7760f20084ade9a09710a47 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:43:38 +0200 Subject: [PATCH 274/409] Fix typo in test case --- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 +- tests/queries/0_stateless/02534_keyed_siphash.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 8b147025a05..a05446a494e 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -239,7 +239,7 @@ Check bug found fuzzing Test arrays and maps 608E1FF030C9E206185B112C2A25F1A7 ABB65AE97711A2E053E324ED88B1D08B -Test emtpy arrays and maps +Test empty arrays and maps 4761183170873013810 0AD04BFD000000000000000000000000 4761183170873013810 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index ba3c4a9156d..7cfc82512bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -346,7 +346,7 @@ INSERT INTO sipHashKeyed_keys FORMAT VALUES ({'a':'b', 'c':'d'}), ({'e':'f', 'g' SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FROM sipHashKeyed_keys ORDER BY a; DROP TABLE sipHashKeyed_keys; -SELECT 'Test emtpy arrays and maps'; +SELECT 'Test empty arrays and maps'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); From 0dc18247df3a290b4fb312325ff3b2a44a3f8357 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 15:10:24 +0000 Subject: [PATCH 275/409] Update version_date.tsv and changelogs after v24.6.3.38-stable --- docs/changelogs/v24.6.3.38-stable.md | 83 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 + 2 files changed, 86 insertions(+) create mode 100644 docs/changelogs/v24.6.3.38-stable.md diff --git a/docs/changelogs/v24.6.3.38-stable.md b/docs/changelogs/v24.6.3.38-stable.md new file mode 100644 index 00000000000..01d7e26e31f --- /dev/null +++ b/docs/changelogs/v24.6.3.38-stable.md @@ -0,0 +1,83 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.3.38-stable (4e33c831589) FIXME as compared to v24.6.2.17-stable (5710a8b5c0c) + +#### Improvement +* Backported in [#66770](https://github.com/ClickHouse/ClickHouse/issues/66770): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#66885](https://github.com/ClickHouse/ClickHouse/issues/66885): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66303](https://github.com/ClickHouse/ClickHouse/issues/66303): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66330](https://github.com/ClickHouse/ClickHouse/issues/66330): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66157](https://github.com/ClickHouse/ClickHouse/issues/66157): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66210](https://github.com/ClickHouse/ClickHouse/issues/66210): Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66456](https://github.com/ClickHouse/ClickHouse/issues/66456): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66228](https://github.com/ClickHouse/ClickHouse/issues/66228): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66183](https://github.com/ClickHouse/ClickHouse/issues/66183): Fix rare case with missing data in the result of distributed query, close [#61432](https://github.com/ClickHouse/ClickHouse/issues/61432). [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Backported in [#66271](https://github.com/ClickHouse/ClickHouse/issues/66271): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66682](https://github.com/ClickHouse/ClickHouse/issues/66682): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66587](https://github.com/ClickHouse/ClickHouse/issues/66587): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66362](https://github.com/ClickHouse/ClickHouse/issues/66362): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68066](https://github.com/ClickHouse/ClickHouse/issues/68066): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68566](https://github.com/ClickHouse/ClickHouse/issues/68566): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68159](https://github.com/ClickHouse/ClickHouse/issues/68159): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#66613](https://github.com/ClickHouse/ClickHouse/issues/66613): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66693](https://github.com/ClickHouse/ClickHouse/issues/66693): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#66577](https://github.com/ClickHouse/ClickHouse/issues/66577): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66721](https://github.com/ClickHouse/ClickHouse/issues/66721): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66670](https://github.com/ClickHouse/ClickHouse/issues/66670): Fix reading of uninitialized memory when hashing empty tuples. This closes [#66559](https://github.com/ClickHouse/ClickHouse/issues/66559). [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#66952](https://github.com/ClickHouse/ClickHouse/issues/66952): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66956](https://github.com/ClickHouse/ClickHouse/issues/66956): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66716](https://github.com/ClickHouse/ClickHouse/issues/66716): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66759](https://github.com/ClickHouse/ClickHouse/issues/66759): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66751](https://github.com/ClickHouse/ClickHouse/issues/66751): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68116](https://github.com/ClickHouse/ClickHouse/issues/68116): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67635](https://github.com/ClickHouse/ClickHouse/issues/67635): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67482](https://github.com/ClickHouse/ClickHouse/issues/67482): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67816](https://github.com/ClickHouse/ClickHouse/issues/67816): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67199](https://github.com/ClickHouse/ClickHouse/issues/67199): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67381](https://github.com/ClickHouse/ClickHouse/issues/67381): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67244](https://github.com/ClickHouse/ClickHouse/issues/67244): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#67503](https://github.com/ClickHouse/ClickHouse/issues/67503): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67887](https://github.com/ClickHouse/ClickHouse/issues/67887): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67578](https://github.com/ClickHouse/ClickHouse/issues/67578): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68611](https://github.com/ClickHouse/ClickHouse/issues/68611): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67852](https://github.com/ClickHouse/ClickHouse/issues/67852): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68275](https://github.com/ClickHouse/ClickHouse/issues/68275): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67808](https://github.com/ClickHouse/ClickHouse/issues/67808): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67838](https://github.com/ClickHouse/ClickHouse/issues/67838): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67993](https://github.com/ClickHouse/ClickHouse/issues/67993): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68208](https://github.com/ClickHouse/ClickHouse/issues/68208): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68093](https://github.com/ClickHouse/ClickHouse/issues/68093): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68124](https://github.com/ClickHouse/ClickHouse/issues/68124): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68221](https://github.com/ClickHouse/ClickHouse/issues/68221): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68173](https://github.com/ClickHouse/ClickHouse/issues/68173): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68339](https://github.com/ClickHouse/ClickHouse/issues/68339): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68396](https://github.com/ClickHouse/ClickHouse/issues/68396): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68668](https://github.com/ClickHouse/ClickHouse/issues/68668): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#66599](https://github.com/ClickHouse/ClickHouse/issues/66599) to 24.6: Fix dropping named collection in local storage"'. [#66922](https://github.com/ClickHouse/ClickHouse/pull/66922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66332](https://github.com/ClickHouse/ClickHouse/issues/66332): Do not raise a NOT_IMPLEMENTED error when getting s3 metrics with a multiple disk configuration. [#65403](https://github.com/ClickHouse/ClickHouse/pull/65403) ([Elena Torró](https://github.com/elenatorro)). +* Backported in [#66142](https://github.com/ClickHouse/ClickHouse/issues/66142): Fix flaky test_storage_s3_queue tests. [#66009](https://github.com/ClickHouse/ClickHouse/pull/66009) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#66389](https://github.com/ClickHouse/ClickHouse/issues/66389): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66428](https://github.com/ClickHouse/ClickHouse/issues/66428): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66546](https://github.com/ClickHouse/ClickHouse/issues/66546): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66861](https://github.com/ClickHouse/ClickHouse/issues/66861): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66877](https://github.com/ClickHouse/ClickHouse/issues/66877): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67061](https://github.com/ClickHouse/ClickHouse/issues/67061): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66940](https://github.com/ClickHouse/ClickHouse/issues/66940): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67254](https://github.com/ClickHouse/ClickHouse/issues/67254): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67414](https://github.com/ClickHouse/ClickHouse/issues/67414): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). +* Update version after release. [#67909](https://github.com/ClickHouse/ClickHouse/pull/67909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68079](https://github.com/ClickHouse/ClickHouse/issues/68079): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..cc168f58862 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,9 +3,11 @@ v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +16,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 0b9c24f31d548c87deca3334282c14fc78a295ba Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 15 Aug 2024 12:09:50 +0000 Subject: [PATCH 276/409] write metadata to disk and keeper in the same format --- src/Storages/ColumnsDescription.cpp | 30 +++++--- src/Storages/ColumnsDescription.h | 6 +- .../__init__.py | 0 .../config/enable_keeper.xml | 26 +++++++ .../config/users.xml | 8 +++ .../test.py | 71 +++++++++++++++++++ 6 files changed, 128 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d724245b49..0212bbd6fff 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -113,7 +113,15 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && ast_to_str(ttl) == ast_to_str(other.ttl); } -void ColumnDescription::writeText(WriteBuffer & buf) const +String formatASTStateAware(IAST & ast, IAST::FormatState & state) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true, false); + ast.formatImpl(settings, state, IAST::FormatStateStacked()); + return buf.str(); +} + +void ColumnDescription::writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const { /// NOTE: Serialization format is insane. @@ -126,20 +134,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeChar('\t', buf); DB::writeText(DB::toString(default_desc.kind), buf); writeChar('\t', buf); - writeEscapedString(queryToString(default_desc.expression), buf); + writeEscapedString(formatASTStateAware(*default_desc.expression, state), buf); } - if (!comment.empty()) + if (!comment.empty() && include_comment) { writeChar('\t', buf); DB::writeText("COMMENT ", buf); - writeEscapedString(queryToString(ASTLiteral(Field(comment))), buf); + auto ast = ASTLiteral(Field(comment)); + writeEscapedString(formatASTStateAware(ast, state), buf); } if (codec) { writeChar('\t', buf); - writeEscapedString(queryToString(codec), buf); + writeEscapedString(formatASTStateAware(*codec, state), buf); } if (!settings.empty()) @@ -150,21 +159,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const ASTSetQuery ast; ast.is_standalone = false; ast.changes = settings; - writeEscapedString(queryToString(ast), buf); + writeEscapedString(formatASTStateAware(ast, state), buf); DB::writeText(")", buf); } if (!statistics.empty()) { writeChar('\t', buf); - writeEscapedString(queryToString(statistics.getAST()), buf); + writeEscapedString(formatASTStateAware(*statistics.getAST(), state), buf); } if (ttl) { writeChar('\t', buf); DB::writeText("TTL ", buf); - writeEscapedString(queryToString(ttl), buf); + writeEscapedString(formatASTStateAware(*ttl, state), buf); } writeChar('\n', buf); @@ -895,16 +904,17 @@ void ColumnsDescription::resetColumnTTLs() } -String ColumnsDescription::toString() const +String ColumnsDescription::toString(bool include_comments) const { WriteBufferFromOwnString buf; + IAST::FormatState ast_format_state; writeCString("columns format version: 1\n", buf); DB::writeText(columns.size(), buf); writeCString(" columns:\n", buf); for (const ColumnDescription & column : columns) - column.writeText(buf); + column.writeText(buf, ast_format_state, include_comments); return buf.str(); } diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index f0760160f0a..c89c26501e8 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -104,7 +104,7 @@ struct ColumnDescription bool operator==(const ColumnDescription & other) const; bool operator!=(const ColumnDescription & other) const { return !(*this == other); } - void writeText(WriteBuffer & buf) const; + void writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const; void readText(ReadBuffer & buf); }; @@ -137,7 +137,7 @@ public: /// NOTE Must correspond with Nested::flatten function. void flattenNested(); /// TODO: remove, insert already flattened Nested columns. - bool operator==(const ColumnsDescription & other) const { return columns == other.columns; } + bool operator==(const ColumnsDescription & other) const { return toString(false) == other.toString(false); } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } auto begin() const { return columns.begin(); } @@ -221,7 +221,7 @@ public: /// Does column has non default specified compression codec bool hasCompressionCodec(const String & column_name) const; - String toString() const; + String toString(bool include_comments = true) const; static ColumnsDescription parse(const String & str); size_t size() const diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml new file mode 100644 index 00000000000..4ca4f604ec3 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml @@ -0,0 +1,26 @@ + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml new file mode 100644 index 00000000000..c5de0b6819c --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml @@ -0,0 +1,8 @@ + + + + default + + + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py new file mode 100644 index 00000000000..e0c15e18c23 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py @@ -0,0 +1,71 @@ +import pytest +import random +import string + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=[ + "config/enable_keeper.xml", + "config/users.xml", + ], + stay_alive=True, + with_minio=True, + macros={"shard": 1, "replica": 1}, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def randomize_table_name(table_name, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{table_name}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + +@pytest.mark.parametrize("engine", ["ReplicatedMergeTree"]) +def test_aliases_in_default_expr_not_break_table_structure(start_cluster, engine): + """ + Making sure that using aliases in columns' default expressions does not lead to having different columns metadata in ZooKeeper and on disk. + Issue: https://github.com/ClickHouse/clickhouse-private/issues/5150 + """ + + data = '{"event": {"col1-key": "col1-val", "col2-key": "col2-val"}}' + + table_name = randomize_table_name("t") + + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ( + `data` String, + `col1` String DEFAULT JSONExtractString(JSONExtractString(data, 'event') AS event, 'col1-key'), + `col2` String MATERIALIZED JSONExtractString(JSONExtractString(data, 'event') AS event, 'col2-key') + ) + ENGINE = {engine}('/test/{table_name}', '{{replica}}') + ORDER BY col1 + """ + ) + + node.restart_clickhouse() + + node.query( + f""" + INSERT INTO {table_name} (data) VALUES ('{data}'); + """ + ) + assert node.query(f"SELECT data FROM {table_name}").strip() == data + assert node.query(f"SELECT col1 FROM {table_name}").strip() == "col1-val" + assert node.query(f"SELECT col2 FROM {table_name}").strip() == "col2-val" + + node.query(f"DROP TABLE {table_name}") From 859d2bfe273f571458be6f007761bc8c743d589a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 17:18:06 +0200 Subject: [PATCH 277/409] move stopFlushThread to SystemLogBase --- src/Common/SystemLogBase.cpp | 19 +++++++++++++++++++ src/Common/SystemLogBase.h | 2 ++ src/Interpreters/PeriodicLog.cpp | 6 +++--- src/Interpreters/PeriodicLog.h | 2 +- src/Interpreters/SystemLog.cpp | 21 +-------------------- src/Interpreters/SystemLog.h | 7 +------ 6 files changed, 27 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 127c8862a35..45f4eb1c5a6 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -273,6 +273,25 @@ void SystemLogBase::startup() saving_thread = std::make_unique([this] { savingThreadFunction(); }); } +template +void SystemLogBase::stopFlushThread() +{ + { + std::lock_guard lock(thread_mutex); + + if (!saving_thread || !saving_thread->joinable()) + return; + + if (is_shutdown) + return; + + is_shutdown = true; + queue->shutdown(); + } + + saving_thread->join(); +} + template void SystemLogBase::add(LogElement element) { diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0d7b04d5c57..0942e920a42 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -216,6 +216,8 @@ public: static consteval bool shouldTurnOffLogger() { return false; } protected: + void stopFlushThread() final; + std::shared_ptr> queue; }; } diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 15970ca5b81..1b285aad3ff 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -11,7 +11,7 @@ void PeriodicLog::startCollect(size_t collect_interval_milliseconds_ { collect_interval_milliseconds = collect_interval_milliseconds_; is_shutdown_metric_thread = false; - flush_thread = std::make_unique([this] { threadFunction(); }); + collecting_thread = std::make_unique([this] { threadFunction(); }); } template @@ -20,8 +20,8 @@ void PeriodicLog::stopCollect() bool old_val = false; if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) return; - if (flush_thread) - flush_thread->join(); + if (collecting_thread) + collecting_thread->join(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index ceac8088d40..8254a02434a 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -36,7 +36,7 @@ protected: private: void threadFunction(); - std::unique_ptr flush_thread; + std::unique_ptr collecting_thread; size_t collect_interval_milliseconds; std::atomic is_shutdown_metric_thread{false}; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 832c39bfaf8..6a3ec197c6e 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -402,32 +402,13 @@ SystemLog::SystemLog( template void SystemLog::shutdown() { - stopFlushThread(); + Base::stopFlushThread(); auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) table->flushAndShutdown(); } -template -void SystemLog::stopFlushThread() -{ - { - std::lock_guard lock(thread_mutex); - - if (!saving_thread || !saving_thread->joinable()) - return; - - if (is_shutdown) - return; - - is_shutdown = true; - queue->shutdown(); - } - - saving_thread->join(); -} - template void SystemLog::savingThreadFunction() diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9e1af3578bd..31652c1af67 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -125,8 +125,6 @@ public: void shutdown() override; - void stopFlushThread() override; - /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. @@ -136,10 +134,7 @@ public: protected: LoggerPtr log; - using ISystemLog::is_shutdown; - using ISystemLog::saving_thread; - using ISystemLog::thread_mutex; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; 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 278/409] 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 51fbc629c6dff4653e687228b0507947516072bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 15:42:17 +0000 Subject: [PATCH 279/409] Update version_date.tsv and changelogs after v24.7.3.47-stable --- docs/changelogs/v24.7.3.47-stable.md | 55 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++ 2 files changed, 59 insertions(+) create mode 100644 docs/changelogs/v24.7.3.47-stable.md diff --git a/docs/changelogs/v24.7.3.47-stable.md b/docs/changelogs/v24.7.3.47-stable.md new file mode 100644 index 00000000000..e5f23a70fe1 --- /dev/null +++ b/docs/changelogs/v24.7.3.47-stable.md @@ -0,0 +1,55 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.3.47-stable (2e50fe27a14) FIXME as compared to v24.7.2.13-stable (6e41f601b2f) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68232](https://github.com/ClickHouse/ClickHouse/issues/68232): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#67969](https://github.com/ClickHouse/ClickHouse/issues/67969): Fixed reading of subcolumns after `ALTER ADD COLUMN` query. [#66243](https://github.com/ClickHouse/ClickHouse/pull/66243) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68068](https://github.com/ClickHouse/ClickHouse/issues/68068): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#67637](https://github.com/ClickHouse/ClickHouse/issues/67637): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67820](https://github.com/ClickHouse/ClickHouse/issues/67820): Fix possible deadlock on query cancel with parallel replicas. [#66905](https://github.com/ClickHouse/ClickHouse/pull/66905) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67818](https://github.com/ClickHouse/ClickHouse/issues/67818): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67766](https://github.com/ClickHouse/ClickHouse/issues/67766): Fix crash of `uniq` and `uniqTheta ` with `tuple()` argument. Closes [#67303](https://github.com/ClickHouse/ClickHouse/issues/67303). [#67306](https://github.com/ClickHouse/ClickHouse/pull/67306) ([flynn](https://github.com/ucasfl)). +* Backported in [#67881](https://github.com/ClickHouse/ClickHouse/issues/67881): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68613](https://github.com/ClickHouse/ClickHouse/issues/68613): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67854](https://github.com/ClickHouse/ClickHouse/issues/67854): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68278](https://github.com/ClickHouse/ClickHouse/issues/68278): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68040](https://github.com/ClickHouse/ClickHouse/issues/68040): Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#68038](https://github.com/ClickHouse/ClickHouse/issues/68038): Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Backported in [#67713](https://github.com/ClickHouse/ClickHouse/issues/67713): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67840](https://github.com/ClickHouse/ClickHouse/issues/67840): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67995](https://github.com/ClickHouse/ClickHouse/issues/67995): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68224](https://github.com/ClickHouse/ClickHouse/issues/68224): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68095](https://github.com/ClickHouse/ClickHouse/issues/68095): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68126](https://github.com/ClickHouse/ClickHouse/issues/68126): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68223](https://github.com/ClickHouse/ClickHouse/issues/68223): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68175](https://github.com/ClickHouse/ClickHouse/issues/68175): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68341](https://github.com/ClickHouse/ClickHouse/issues/68341): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68398](https://github.com/ClickHouse/ClickHouse/issues/68398): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68669](https://github.com/ClickHouse/ClickHouse/issues/68669): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67518](https://github.com/ClickHouse/ClickHouse/issues/67518): Split slow test 03036_dynamic_read_subcolumns. [#66954](https://github.com/ClickHouse/ClickHouse/pull/66954) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67516](https://github.com/ClickHouse/ClickHouse/issues/67516): Split 01508_partition_pruning_long. [#66983](https://github.com/ClickHouse/ClickHouse/pull/66983) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67529](https://github.com/ClickHouse/ClickHouse/issues/67529): Reduce max time of 00763_long_lock_buffer_alter_destination_table. [#67185](https://github.com/ClickHouse/ClickHouse/pull/67185) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67803](https://github.com/ClickHouse/ClickHouse/issues/67803): Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67643](https://github.com/ClickHouse/ClickHouse/issues/67643): [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. [#67506](https://github.com/ClickHouse/ClickHouse/pull/67506) ([Alexey Katsman](https://github.com/alexkats)). +* Backported in [#67609](https://github.com/ClickHouse/ClickHouse/issues/67609): Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. [#67590](https://github.com/ClickHouse/ClickHouse/pull/67590) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67871](https://github.com/ClickHouse/ClickHouse/issues/67871): Fix 02434_cancel_insert_when_client_dies. [#67600](https://github.com/ClickHouse/ClickHouse/pull/67600) ([vdimir](https://github.com/vdimir)). +* Backported in [#67704](https://github.com/ClickHouse/ClickHouse/issues/67704): Fix 02910_bad_logs_level_in_local in fast tests. [#67603](https://github.com/ClickHouse/ClickHouse/pull/67603) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67689](https://github.com/ClickHouse/ClickHouse/issues/67689): Fix 01605_adaptive_granularity_block_borders. [#67605](https://github.com/ClickHouse/ClickHouse/pull/67605) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67827](https://github.com/ClickHouse/ClickHouse/issues/67827): Try fix 03143_asof_join_ddb_long. [#67620](https://github.com/ClickHouse/ClickHouse/pull/67620) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67892](https://github.com/ClickHouse/ClickHouse/issues/67892): Revert "Merge pull request [#66510](https://github.com/ClickHouse/ClickHouse/issues/66510) from canhld94/fix_trivial_count_non_deterministic_func". [#67800](https://github.com/ClickHouse/ClickHouse/pull/67800) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68081](https://github.com/ClickHouse/ClickHouse/issues/68081): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Update version after release. [#68044](https://github.com/ClickHouse/ClickHouse/pull/68044) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68269](https://github.com/ClickHouse/ClickHouse/issues/68269): [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#68432](https://github.com/ClickHouse/ClickHouse/issues/68432): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68538](https://github.com/ClickHouse/ClickHouse/issues/68538): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68555](https://github.com/ClickHouse/ClickHouse/issues/68555): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..6ef5ace4ba6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,14 @@ v24.8.1.2684-lts 2024-08-21 +v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +17,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 5f61e193401c5fa46db03542cb88ba4188ed00e9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:46:47 +0200 Subject: [PATCH 280/409] small fixes --- docs/ru/getting-started/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 4a0ec258c64..5bce41ec07a 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -22,7 +22,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su ### Из deb-пакетов {#install-from-deb-packages} -Яндекс рекомендует использовать официальные скомпилированные `deb`-пакеты для Debian или Ubuntu. Для установки пакетов выполните: +Рекомендуется использовать официальные скомпилированные `deb`-пакеты для Debian или Ubuntu. Для установки пакетов выполните: ``` bash sudo apt-get install -y apt-transport-https ca-certificates curl gnupg @@ -55,7 +55,7 @@ clickhouse-client # or "clickhouse-client --password" if you've set up a passwor ::: ### Из rpm-пакетов {#from-rpm-packages} -Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm`-пакеты для CentOS, RedHat и всех остальных дистрибутивов Linux, основанных на rpm. +Команда ClickHouse рекомендует использовать официальные предкомпилированные `rpm`-пакеты для CentOS, RedHat и всех остальных дистрибутивов Linux, основанных на rpm. #### Установка официального репозитория @@ -102,7 +102,7 @@ sudo yum install clickhouse-server clickhouse-client ### Из tgz-архивов {#from-tgz-archives} -Команда ClickHouse в Яндексе рекомендует использовать предкомпилированные бинарники из `tgz`-архивов для всех дистрибутивов, где невозможна установка `deb`- и `rpm`- пакетов. +Команда ClickHouse рекомендует использовать предкомпилированные бинарники из `tgz`-архивов для всех дистрибутивов, где невозможна установка `deb`- и `rpm`- пакетов. Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://packages.clickhouse.com/tgz/. После этого архивы нужно распаковать и воспользоваться скриптами установки. Пример установки самой свежей версии: From 980b02bfd67defbbdf78165e8225fb754d722d7a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:48:57 +0200 Subject: [PATCH 281/409] fix compatibility with en version --- docs/ru/getting-started/install.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 5bce41ec07a..f8a660fbec9 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -31,9 +31,17 @@ curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | s echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update +``` +#### Установка ClickHouse server и client + +```bash sudo apt-get install -y clickhouse-server clickhouse-client +``` +#### Запуск ClickHouse server + +```bash sudo service clickhouse-server start clickhouse-client # or "clickhouse-client --password" if you've set up a password. ``` From 7c3a013d56c1dbd5b72f04f6be61f007004aaefa Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Thu, 22 Aug 2024 16:53:30 +0100 Subject: [PATCH 282/409] Update newjson.md --- docs/en/sql-reference/data-types/newjson.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 9e43216df6c..f7fc7e1498e 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -70,7 +70,7 @@ SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json └────────────────────────────────────────────────┘ ``` -CAST from named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. +CAST from `JSON`, named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. ## Reading JSON paths as subcolumns From 28fbd8a4eff4eafa7db99eb37e38376ffda11763 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:56:16 +0200 Subject: [PATCH 283/409] fix stateless tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 2 -- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index acdadc2510b..a9d856babce 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -34,8 +34,6 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 -4081 -2070 2070 b 1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index b3d196924af..6734c5f14ad 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -32,7 +32,7 @@ SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMI $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From 9c0e1df1663dd5c56066dd615fc3cafe6408d308 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:58:15 +0200 Subject: [PATCH 284/409] Fix flaky test 00989_parallel_parts_loading --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index 407e124f137..dc074241ff6 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings +-- small insert block size can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From 0bd8ebf62616ce882b0ebc46945c837a5a91ba44 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 22 Aug 2024 17:58:56 +0200 Subject: [PATCH 285/409] Update README.md adding community call. resolving recent recordings --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c9474ef0fc0..9099fd48659 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 20 +* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 266 ## Upcoming Events @@ -58,7 +58,7 @@ Other upcoming meetups ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now! +* **Recording available**: [**v24.8 LTS Release Call**](https://www.youtube.com/watch?v=AeLmp2jc51k) All the features of 24.8 LTS, one convenient video! Watch it now! ## Interested in joining ClickHouse and making it your full-time job? From 52cdd88eb6d7bbb5d395dd80445655ad47c83c92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:59:10 +0200 Subject: [PATCH 286/409] Better comment --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index dc074241ff6..3b73e6a0e3c 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,5 +1,5 @@ -- Tags: no-random-settings, no-random-merge-tree-settings --- small insert block size can make insert terribly slow, especially with some build like msan +-- small number of insert threads can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From e7b89537bf1bb760c6082f04de4668bd1c00f33a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 18:02:42 +0200 Subject: [PATCH 287/409] fix style --- src/Interpreters/PeriodicLog.cpp | 1 - src/Interpreters/SystemLog.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 1b285aad3ff..22bc14856c4 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Functions/DateTimeTransforms.h" namespace DB { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 31652c1af67..c03f9370068 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -134,7 +134,7 @@ public: protected: LoggerPtr log; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; From 1692360233593e635c5a7797847bdfd8a0ffa33e Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 22 Aug 2024 18:12:38 +0200 Subject: [PATCH 288/409] Update README.md 26 and 266 are different --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 9099fd48659..83a5c05c667 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 266 +* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 26 ## Upcoming Events From 4264fbc037accedecebcd8122910e4406e92cd58 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 16:16:47 +0000 Subject: [PATCH 289/409] Update version_date.tsv and changelogs after v24.8.2.3-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.2.3-lts.md | 12 ++++++++++++ utils/list-versions/version_date.tsv | 5 +++++ 5 files changed, 20 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.8.2.3-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index fc93cee5bbc..6ff7ea43374 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3ceaf2a08b4..c87885d3b49 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 76db997821c..6ccf74823e2 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.2.3-lts.md b/docs/changelogs/v24.8.2.3-lts.md new file mode 100644 index 00000000000..69dfc9961a2 --- /dev/null +++ b/docs/changelogs/v24.8.2.3-lts.md @@ -0,0 +1,12 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.2.3-lts (b54f79ed323) FIXME as compared to v24.8.1.2684-lts (161c62fd295) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68670](https://github.com/ClickHouse/ClickHouse/issues/68670): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..199c4f822f4 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,15 @@ +v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +18,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From fa453c3664b18da7a6945e662b881f80fedadc5b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 18:13:45 +0200 Subject: [PATCH 290/409] Disable SqlLogic job --- tests/ci/ci_config.py | 7 ++++--- tests/ci/ci_definitions.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 58de25f039f..0885f1d9ec2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -498,9 +498,10 @@ class CI: JobNames.SQLANCER_DEBUG: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], ), - JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], - ), + # TODO: job does not work at all, uncomment and fix + # JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( + # required_builds=[BuildNames.PACKAGE_RELEASE], + # ), JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 1cdb3f1487e..9d95a19790f 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -204,7 +204,7 @@ class JobNames(metaclass=WithIter): PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" - SQL_LOGIC_TEST = "Sqllogic test (release)" + # SQL_LOGIC_TEST = "Sqllogic test (release)" SQLANCER = "SQLancer (release)" SQLANCER_DEBUG = "SQLancer (debug)" From ad60876777fdfe178cae9184a89603376c7d2260 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 16:46:41 +0000 Subject: [PATCH 291/409] support query paramters in async inserts --- src/Interpreters/AsynchronousInsertQueue.cpp | 5 +++ src/Interpreters/AsynchronousInsertQueue.h | 1 + .../Executors/StreamingFormatExecutor.cpp | 14 ++++++++ .../Executors/StreamingFormatExecutor.h | 3 ++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 10 ++++++ .../Formats/Impl/ValuesBlockInputFormat.h | 1 + .../03228_async_insert_query_params.reference | 13 +++++++ .../03228_async_insert_query_params.sh | 36 +++++++++++++++++++ 8 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params.reference create mode 100755 tests/queries/0_stateless/03228_async_insert_query_params.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..5e012d179c8 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -389,6 +389,10 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr if (data_kind == DataKind::Preprocessed) insert_query.format = "Native"; + /// Query parameters make sense only for format Values. + if (insert_query.format == "Values") + entry->query_parameters = query_context->getQueryParameters(); + InsertQuery key{query, query_context->getUserID(), query_context->getCurrentRoles(), settings, data_kind}; InsertDataPtr data_to_process; std::future insert_future; @@ -999,6 +1003,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( "Expected entry with data kind Parsed. Got: {}", entry->chunk.getDataKind()); auto buffer = std::make_unique(*bytes); + executor.setQueryParameters(entry->query_parameters); size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index da14b43d276..17607ac1879 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -147,6 +147,7 @@ private: const String format; MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; + NameToNameMap query_parameters; Entry( DataChunk && chunk_, diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 12dd685a735..5b35ac6020d 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -32,6 +33,19 @@ MutableColumns StreamingFormatExecutor::getResultColumns() return ret_columns; } +void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters) +{ + if (parameters.empty()) + return; + + /// Query parameters make sense only for format Values. + auto * values_format = typeid_cast(format.get()); + if (!values_format) + return; + + values_format->setQueryParameters(parameters); +} + size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) { format->setReadBuffer(buffer); diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index 3aa90ab0360..f159178df8c 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -39,6 +39,9 @@ public: /// Releases currently accumulated columns. MutableColumns getResultColumns(); + /// Sets query parameters for input format if applicable. + void setQueryParameters(const NameToNameMap & parameters); + private: const Block header; const InputFormatPtr format; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 9839f64b947..e34000df5f4 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -663,6 +663,16 @@ void ValuesBlockInputFormat::resetReadBuffer() IInputFormat::resetReadBuffer(); } +void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters) +{ + if (parameters.empty()) + return; + + auto context_copy = Context::createCopy(context); + context_copy->setQueryParameters(parameters); + context = std::move(context_copy); +} + ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : IRowSchemaReader(buf, format_settings_), buf(in_) { diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 0abafc896ff..228e8d0d572 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -38,6 +38,7 @@ public: /// TODO: remove context somehow. void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); } + void setQueryParameters(const NameToNameMap & parameters); const BlockMissingValues & getMissingValues() const override { return block_missing_values; } diff --git a/tests/queries/0_stateless/03228_async_insert_query_params.reference b/tests/queries/0_stateless/03228_async_insert_query_params.reference new file mode 100644 index 00000000000..9de99658a53 --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params.reference @@ -0,0 +1,13 @@ +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 diff --git a/tests/queries/0_stateless/03228_async_insert_query_params.sh b/tests/queries/0_stateless/03228_async_insert_query_params.sh new file mode 100755 index 00000000000..6b130389220 --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS t_async_insert_params; + CREATE TABLE t_async_insert_params (id UInt64) ENGINE = Memory; +" +cmd_params="--async_insert 1 --async_insert_busy_timeout_max_ms 300000 --async_insert_busy_timeout_min_ms 300000 --wait_for_async_insert 0 --async_insert_use_adaptive_busy_timeout 0" + +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p1 = 11; INSERT INTO t_async_insert_params VALUES ({p1:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 12; INSERT INTO t_async_insert_params VALUES ({p2:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 1000; INSERT INTO t_async_insert_params VALUES (13);" +$CLICKHOUSE_CLIENT $cmd_params -q 'SET param_p2 = 1000; INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 14};' + +$CLICKHOUSE_CLIENT $cmd_params --param_p1 15 -q "INSERT INTO t_async_insert_params VALUES ({p1:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 16 -q "INSERT INTO t_async_insert_params VALUES ({p2:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q "INSERT INTO t_async_insert_params VALUES (17);" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 18};' + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0" + +${CLICKHOUSE_CURL} -sS "$url¶m_p1=19" -d "INSERT INTO t_async_insert_params VALUES ({p1:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=20" -d "INSERT INTO t_async_insert_params VALUES ({p2:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p3=21" -d "INSERT INTO t_async_insert_params VALUES ({p3:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=1000" -d "INSERT INTO t_async_insert_params VALUES (22)" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=1000" -d 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 23}' + +$CLICKHOUSE_CLIENT -q " + SYSTEM FLUSH ASYNC INSERT QUEUE; + SELECT id FROM t_async_insert_params ORDER BY id; + DROP TABLE IF EXISTS t_async_insert_params; +" From 06c46ee75bcb94fe02ac68df6a4a044145804d76 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 18:56:50 +0200 Subject: [PATCH 292/409] add one more test --- .../0_stateless/03203_hive_style_partitioning.reference | 1 + tests/queries/0_stateless/03203_hive_style_partitioning.sh | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index a9d856babce..0fbc1fb556e 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -37,6 +37,7 @@ Array(Int64) LowCardinality(Float64) 2070 b 1 +1 TESTING THE URL PARTITIONING last Elizabeth Frank Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 6734c5f14ad..8ab18f5edfe 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -29,6 +29,12 @@ SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.c SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "INCORRECT_DATA" + $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; From 2a32207e9ee44d52d6fbca7313d847b4eef1c4fb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 293/409] 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 294/409] 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 295/409] 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 de2f1adf9167db66469efd8e8b5d2f828f993ec1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2024 18:02:03 +0000 Subject: [PATCH 296/409] Fix mergine of aggregated data for grouping sets. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 - src/Planner/Planner.cpp | 2 - .../QueryPlan/MergingAggregatedStep.cpp | 2 +- .../Transforms/MergingAggregatedTransform.cpp | 118 ++++++++++++++++-- .../Transforms/MergingAggregatedTransform.h | 10 +- .../02165_replicated_grouping_sets.reference | 54 ++++++++ .../02165_replicated_grouping_sets.sql | 5 + 7 files changed, 179 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0c79f4310ce..9e5fffac6e4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2010,8 +2010,6 @@ static void executeMergeAggregatedImpl( SortDescription group_by_sort_description) { auto keys = aggregation_keys.getNames(); - if (has_grouping_sets) - keys.insert(keys.begin(), "__grouping_set"); /** There are two modes of distributed aggregation. * diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d3d20c6fba0..c0efed8550f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -504,8 +504,6 @@ void addMergingAggregatedStep(QueryPlan & query_plan, */ auto keys = aggregation_analysis_result.aggregation_keys; - if (!aggregation_analysis_result.grouping_sets_parameters_list.empty()) - keys.insert(keys.begin(), "__grouping_set"); Aggregator::Params params(keys, aggregation_analysis_result.aggregate_descriptions, diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index a5062ac8216..50bd1a882ef 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -48,7 +48,7 @@ MergingAggregatedStep::MergingAggregatedStep( bool memory_bound_merging_of_aggregation_results_enabled_) : ITransformingStep( input_stream_, - params_.getHeader(input_stream_.header, final_), + MergingAggregatedTransform::appendGroupingIfNeeded(input_stream_.header, params_.getHeader(input_stream_.header, final_)), getTraits(should_produce_results_in_order_of_bucket_number_)) , params(std::move(params_)) , final(final_) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 446e60a0b81..114a32b3d83 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -10,13 +11,106 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) +{ + if (in_header.has("__grouping_set")) + out_header.insert(0, in_header.getByName("__grouping_set")); + + return out_header; +} + MergingAggregatedTransform::MergingAggregatedTransform( Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_) - : IAccumulatingTransform(std::move(header_), params_->getHeader()) - , params(std::move(params_)), max_threads(max_threads_) + : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params_->getHeader())) + , params(std::move(params_)), max_threads(max_threads_), has_grouping_sets(header_.has("__grouping_set")) { } +void MergingAggregatedTransform::addBlock(Block block) +{ + if (!has_grouping_sets) + { + auto & bucket_to_blocks = grouping_sets[0]; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + return; + } + + auto grouping_position = block.getPositionByName("__grouping_set"); + auto grouping_column = block.getByPosition(grouping_position).column; + block.erase(grouping_position); + + const auto * grouping_column_typed = typeid_cast(grouping_column.get()); + if (!grouping_column_typed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); + + const auto & grouping_data = grouping_column_typed->getData(); + std::map enumerated_groups; + IColumn::Selector selector; + + size_t num_rows = grouping_data.size(); + UInt64 last_group = grouping_data[0]; + for (size_t row = 1; row < num_rows; ++row) + { + auto group = grouping_data[row]; + if (last_group == group) + continue; + + if (enumerated_groups.empty()) + { + selector.reserve(num_rows); + enumerated_groups.emplace(last_group, enumerated_groups.size()); + } + + selector.resize_fill(row, enumerated_groups[last_group]); + enumerated_groups.emplace(last_group, enumerated_groups.size()); + } + + if (enumerated_groups.empty()) + { + auto & bucket_to_blocks = grouping_sets[last_group]; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + return; + } + + selector.resize_fill(num_rows, enumerated_groups[last_group]); + + const size_t num_groups = enumerated_groups.size(); + Blocks splitted_blocks(num_groups); + + for (size_t group_id = 0; group_id < num_groups; ++group_id) + splitted_blocks[group_id] = block.cloneEmpty(); + + size_t columns_in_block = block.columns(); + for (size_t col_idx_in_block = 0; col_idx_in_block < columns_in_block; ++col_idx_in_block) + { + MutableColumns splitted_columns = block.getByPosition(col_idx_in_block).column->scatter(num_groups, selector); + for (size_t group_id = 0; group_id < num_groups; ++group_id) + splitted_blocks[group_id].getByPosition(col_idx_in_block).column = std::move(splitted_columns[group_id]); + } + + for (auto [group, group_id] : enumerated_groups) + { + auto & bucket_to_blocks = grouping_sets[group]; + auto & splitted_block = splitted_blocks[group_id]; + splitted_block.info = block.info; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); + } +} + +void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) +{ + auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); + for (auto & block : block_list) + { + auto num_rows = block.rows(); + ColumnWithTypeAndName col; + col.type = std::make_shared(); + col.name = "__grouping_set"; + col.column = ColumnUInt64::create(num_rows, group); + block.insert(grouping_position, std::move(col)); + } +} + void MergingAggregatedTransform::consume(Chunk chunk) { if (!consume_started) @@ -46,7 +140,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) block.info.is_overflows = agg_info->is_overflows; block.info.bucket_num = agg_info->bucket_num; - bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block)); + addBlock(std::move(block)); } else if (chunk.getChunkInfos().get()) { @@ -54,7 +148,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) block.info.is_overflows = false; block.info.bucket_num = -1; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + addBlock(std::move(block)); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); @@ -70,9 +164,19 @@ Chunk MergingAggregatedTransform::generate() /// Exception safety. Make iterator valid in case any method below throws. next_block = blocks.begin(); - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. - params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); - blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); + for (auto & [group, group_blocks] : grouping_sets) + { + /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + AggregatedDataVariants data_variants; + params->aggregator.mergeBlocks(std::move(group_blocks), data_variants, max_threads, is_cancelled); + auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); + + if (has_grouping_sets) + appendGroupingColumn(group, merged_blocks); + + blocks.splice(blocks.end(), std::move(merged_blocks)); + } + next_block = blocks.begin(); } diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index ade76b2f304..1d801f7a94d 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -15,6 +15,8 @@ public: MergingAggregatedTransform(Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_); String getName() const override { return "MergingAggregatedTransform"; } + static Block appendGroupingIfNeeded(const Block & in_header, Block out_header); + protected: void consume(Chunk chunk) override; Chunk generate() override; @@ -24,8 +26,9 @@ private: LoggerPtr log = getLogger("MergingAggregatedTransform"); size_t max_threads; - AggregatedDataVariants data_variants; - Aggregator::BucketToBlocks bucket_to_blocks; + using GroupingSets = std::unordered_map; + GroupingSets grouping_sets; + const bool has_grouping_sets; UInt64 total_input_rows = 0; UInt64 total_input_blocks = 0; @@ -35,6 +38,9 @@ private: bool consume_started = false; bool generate_started = false; + + void addBlock(Block block); + void appendGroupingColumn(UInt64 group, BlocksList & block_list); }; } diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference index 659cd98368d..4589dc7d7a5 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference @@ -11,3 +11,57 @@ 0 6 4 1 10 4 2 14 4 +['.'] +['.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +['.'] +['.'] +['.','.'] +['.','.'] +['.','.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +1 +2 +3 +4 +5 +6 +7 +8 +9 +1 +1 +2 +2 +3 +3 +4 +4 +5 +5 +6 +6 +7 +7 +8 +8 +9 +9 diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql index d92d92c3e72..333dab79575 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql @@ -43,3 +43,8 @@ GROUP BY ORDER BY sum_value ASC, count_value ASC; + +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; From 0e136ded28dc1191dd344500d031f43d7a5750e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2024 19:06:04 +0000 Subject: [PATCH 297/409] Fixing header. --- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 50bd1a882ef..8332ad73df6 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -154,7 +154,9 @@ void MergingAggregatedStep::describeActions(JSONBuilder::JSONMap & map) const void MergingAggregatedStep::updateOutputStream() { - output_stream = createOutputStream(input_streams.front(), params.getHeader(input_streams.front().header, final), getDataStreamTraits()); + const auto & in_header = input_streams.front().header; + output_stream = createOutputStream(input_streams.front(), + MergingAggregatedTransform::appendGroupingIfNeeded(in_header, params.getHeader(in_header, final)), getDataStreamTraits()); if (is_order_overwritten) /// overwrite order again applyOrder(group_by_sort_description, overwritten_sort_scope); } From f89193fa416cc333f549d72bb8ba453907edc951 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 19:12:19 +0000 Subject: [PATCH 298/409] Update version_date.tsv and changelogs after v24.5.5.41-stable --- utils/list-versions/version_date.tsv | 2 -- 1 file changed, 2 deletions(-) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 199c4f822f4..0e25f8d3b62 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,9 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 -v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 -v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 From 4200b3d5cbbfe065073c40f1e122c44189f3554f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 14:02:25 +0200 Subject: [PATCH 299/409] CI: Stress test fix --- tests/clickhouse-test | 2 +- tests/docker_scripts/stress_runner.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4f9380d6f20..ad6173065fe 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3567,7 +3567,7 @@ if __name__ == "__main__": f"Cannot access the specified directory with queries ({args.queries})", file=sys.stderr, ) - sys.exit(1) + assert False, "No --queries provided" CAPTURE_CLIENT_STACKTRACE = args.capture_client_stacktrace diff --git a/tests/docker_scripts/stress_runner.sh b/tests/docker_scripts/stress_runner.sh index 7666398e10b..039c60c8e4e 100755 --- a/tests/docker_scripts/stress_runner.sh +++ b/tests/docker_scripts/stress_runner.sh @@ -10,8 +10,7 @@ dmesg --clear # shellcheck disable=SC1091 source /setup_export_logs.sh -ln -s /repo/tests/clickhouse-test/ci/stress.py /usr/bin/stress -ln -s /repo/tests/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib @@ -266,6 +265,7 @@ fi start_server +cd /repo/tests/ || exit 1 # clickhouse-test can find queries dir from there python3 /repo/tests/ci/stress.py --hung-check --drop-databases --output-folder /test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ && echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \ || echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv From 69f6ea5083f1686becce4ca9fcf47d1404f2d3ed Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 22:07:02 +0200 Subject: [PATCH 300/409] 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 dc862b1411884a462bba8dcf86a474ccbe57e380 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:40:18 +0200 Subject: [PATCH 301/409] fix test --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 8ab18f5edfe..60e8a6e9faa 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -32,7 +32,7 @@ SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMI $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "INCORRECT_DATA" $CLICKHOUSE_LOCAL -n -q """ From 4c790999eb6ad74e3a8f99c072dcc12c956a63d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:18:26 +0200 Subject: [PATCH 302/409] CI: Force package_debug build on release branches --- .github/workflows/release_branches.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 82826794ea3..ec119b6ff95 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -130,6 +130,7 @@ jobs: with: build_name: package_debug data: ${{ needs.RunConfig.outputs.data }} + force: true BuilderBinDarwin: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} From f5739dfe06db8610818fafb5c3a2c33f59fd0a8d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:51:27 +0200 Subject: [PATCH 303/409] CI: Make job rerun possible if triggered manually --- tests/ci/ci.py | 7 +++++-- tests/ci/ci_utils.py | 5 +++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index a9ae078b449..d201b6602f5 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -333,7 +333,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info): CI.JobNames.BUILD_CHECK, ): # we might want to rerun build report job rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name)) - if rerun_helper.is_already_finished_by_status(): + if ( + rerun_helper.is_already_finished_by_status() + and not Utils.is_job_triggered_manually() + ): print("WARNING: Rerunning job with GH status ") status = rerun_helper.get_finished_status() assert status @@ -344,7 +347,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): skip_status = status.state # ci cache check - if not to_be_skipped and not no_cache: + if not to_be_skipped and not no_cache and not Utils.is_job_triggered_manually(): ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() job_config = CI.get_job_config(job_name) if ci_cache.is_successful( diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index a4c0977f47c..e8d9e7dc254 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -18,6 +18,7 @@ class Envs: ) S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") + GITHUB_ACTOR = os.getenv("GITHUB_ACTOR", "") class WithIter(type): @@ -282,3 +283,7 @@ class Utils: ): res = res.replace(*r) return res + + @staticmethod + def is_job_triggered_manually(): + return "robot" not in Envs.GITHUB_ACTOR From 073ef13e36e46cf6f34c0f1cee2dd5212753fd74 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 01:02:46 +0000 Subject: [PATCH 304/409] fix --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 1845919890c..2750bdf5c4b 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -1,3 +1,5 @@ +-- Tags: long, no-parallel + DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 From 60e4bcbbf0b1991b42bcab4b83e55be344e8a659 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Thu, 22 Aug 2024 20:45:28 -0700 Subject: [PATCH 305/409] Update README.md Update Raleigh meetup link --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 83a5c05c667..546f08afd3d 100644 --- a/README.md +++ b/README.md @@ -44,7 +44,7 @@ The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 * [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 * [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 From e5380806653f8d391c6e88664b0096c3c51240f5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:09:03 +0000 Subject: [PATCH 306/409] Update version_date.tsv and changelogs after v24.5.6.45-stable --- docs/changelogs/v24.5.6.45-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 +- 2 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.5.6.45-stable.md diff --git a/docs/changelogs/v24.5.6.45-stable.md b/docs/changelogs/v24.5.6.45-stable.md new file mode 100644 index 00000000000..b329ebab27b --- /dev/null +++ b/docs/changelogs/v24.5.6.45-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.6.45-stable (bdca8604c29) FIXME as compared to v24.5.5.78-stable (0138248cb62) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67902](https://github.com/ClickHouse/ClickHouse/issues/67902): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68252](https://github.com/ClickHouse/ClickHouse/issues/68252): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68064](https://github.com/ClickHouse/ClickHouse/issues/68064): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68158](https://github.com/ClickHouse/ClickHouse/issues/68158): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68115](https://github.com/ClickHouse/ClickHouse/issues/68115): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67886](https://github.com/ClickHouse/ClickHouse/issues/67886): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68272](https://github.com/ClickHouse/ClickHouse/issues/68272): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67807](https://github.com/ClickHouse/ClickHouse/issues/67807): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67836](https://github.com/ClickHouse/ClickHouse/issues/67836): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67991](https://github.com/ClickHouse/ClickHouse/issues/67991): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68207](https://github.com/ClickHouse/ClickHouse/issues/68207): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68091](https://github.com/ClickHouse/ClickHouse/issues/68091): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68122](https://github.com/ClickHouse/ClickHouse/issues/68122): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68171](https://github.com/ClickHouse/ClickHouse/issues/68171): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68337](https://github.com/ClickHouse/ClickHouse/issues/68337): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68667](https://github.com/ClickHouse/ClickHouse/issues/68667): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67862](https://github.com/ClickHouse/ClickHouse/pull/67862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68077](https://github.com/ClickHouse/ClickHouse/issues/68077): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68756](https://github.com/ClickHouse/ClickHouse/issues/68756): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..57a59d7ac49 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,8 +6,8 @@ v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From 1dc7d37ff7ed295be3ef037bf9b1d04b377892b0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 12:30:25 +0000 Subject: [PATCH 307/409] Bump usearch to v2.14.0 --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index e21a5778a0d..7a8967cb442 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 +Subproject commit 7a8967cb442b08ca20c3dd781414378e65957d37 From c40c8b7adb28403d8020282595ddaa72953acbf6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 14:11:33 +0000 Subject: [PATCH 308/409] Enable bf16 + f64 quantization, make bf16 the default --- .../mergetree-family/annindexes.md | 2 +- .../MergeTreeIndexVectorSimilarity.cpp | 4 ++- .../02354_vector_search_queries.reference | 36 +++++++++++++++++++ .../02354_vector_search_queries.sql | 32 +++++++++++++++++ 4 files changed, 72 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 1057ccb5fee..4cf558fc872 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -54,7 +54,7 @@ Parameters: - `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a line between two points in Euclidean space), or `cosineDistance` (the [cosine distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors). -- `quantization`: either `f32`, `f16`, or `i8` for storing the vector with reduced precision (optional, default: `f32`) +- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`) - `m`: the number of neighbors per graph node (optional, default: 16) - `ef_construction`: (optional, default: 128) - `ef_search`: (optional, default: 64) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 4c0da28c3c4..ae183d74782 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -52,8 +52,10 @@ const std::unordered_map distanceFunctionT /// Maps from user-facing name to internal name const std::unordered_map quantizationToScalarKind = { + {"f64", unum::usearch::scalar_kind_t::f64_k}, {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, + {"bf16", unum::usearch::scalar_kind_t::bf16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; /// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported across all distance functions. @@ -461,7 +463,7 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { /// Default parameters: unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet()); - unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; + unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::bf16_k; UsearchHnswParams usearch_hnsw_params; /// Optional parameters: diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index cb3a8c801b1..faff306ef60 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -56,6 +56,24 @@ Expression (Projection) 1 [2,3.2] 2.3323807824711897 2 [4.2,3.4] 4.427188573446585 0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_f64) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -92,6 +110,24 @@ Expression (Projection) 1 [2,3.2] 2.3323807824711897 2 [4.2,3.4] 4.427188573446585 0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_bf16) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index fbf8427d8fe..17939992165 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -75,13 +75,30 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind DROP TABLE tab; SELECT '-- Non-default quantization'; +CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +INSERT INTO tab_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f64 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f64 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab_f32 @@ -108,6 +125,19 @@ FROM tab_f16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_bf16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_bf16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab_i8 @@ -121,8 +151,10 @@ FROM tab_i8 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; +DROP TABLE tab_f64; DROP TABLE tab_f32; DROP TABLE tab_f16; +DROP TABLE tab_bf16; DROP TABLE tab_i8; SELECT '-- Index on Array(Float64) column'; From e1a7bd9163bebf0aeab12d8dd46c729f73b068be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:37:32 +0000 Subject: [PATCH 309/409] Update version_date.tsv and changelogs after v24.6.4.42-stable --- docs/changelogs/v24.6.4.42-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 ++- 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.6.4.42-stable.md diff --git a/docs/changelogs/v24.6.4.42-stable.md b/docs/changelogs/v24.6.4.42-stable.md new file mode 100644 index 00000000000..29b6ba095af --- /dev/null +++ b/docs/changelogs/v24.6.4.42-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.4.42-stable (c534bb4b4dd) FIXME as compared to v24.6.3.95-stable (8325c920d11) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68066](https://github.com/ClickHouse/ClickHouse/issues/68066): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68566](https://github.com/ClickHouse/ClickHouse/issues/68566): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68159](https://github.com/ClickHouse/ClickHouse/issues/68159): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68116](https://github.com/ClickHouse/ClickHouse/issues/68116): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67887](https://github.com/ClickHouse/ClickHouse/issues/67887): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68611](https://github.com/ClickHouse/ClickHouse/issues/68611): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68275](https://github.com/ClickHouse/ClickHouse/issues/68275): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67993](https://github.com/ClickHouse/ClickHouse/issues/67993): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68208](https://github.com/ClickHouse/ClickHouse/issues/68208): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68093](https://github.com/ClickHouse/ClickHouse/issues/68093): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68124](https://github.com/ClickHouse/ClickHouse/issues/68124): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68221](https://github.com/ClickHouse/ClickHouse/issues/68221): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68173](https://github.com/ClickHouse/ClickHouse/issues/68173): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68339](https://github.com/ClickHouse/ClickHouse/issues/68339): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68396](https://github.com/ClickHouse/ClickHouse/issues/68396): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68668](https://github.com/ClickHouse/ClickHouse/issues/68668): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67909](https://github.com/ClickHouse/ClickHouse/pull/67909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68079](https://github.com/ClickHouse/ClickHouse/issues/68079): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68758](https://github.com/ClickHouse/ClickHouse/issues/68758): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..8ce510f110d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,11 +3,12 @@ v24.8.1.2684-lts 2024-08-21 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From eec720dab60ea63b033919bbc4c1f6837920a42d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 08:05:27 +0000 Subject: [PATCH 310/409] Update version_date.tsv and changelogs after v24.7.4.51-stable --- docs/changelogs/v24.7.4.51-stable.md | 36 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 38 insertions(+) create mode 100644 docs/changelogs/v24.7.4.51-stable.md diff --git a/docs/changelogs/v24.7.4.51-stable.md b/docs/changelogs/v24.7.4.51-stable.md new file mode 100644 index 00000000000..a7cf9790383 --- /dev/null +++ b/docs/changelogs/v24.7.4.51-stable.md @@ -0,0 +1,36 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.4.51-stable (70fe2f6fa52) FIXME as compared to v24.7.3.42-stable (63730bc4293) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68232](https://github.com/ClickHouse/ClickHouse/issues/68232): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68068](https://github.com/ClickHouse/ClickHouse/issues/68068): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68613](https://github.com/ClickHouse/ClickHouse/issues/68613): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68278](https://github.com/ClickHouse/ClickHouse/issues/68278): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68040](https://github.com/ClickHouse/ClickHouse/issues/68040): Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#68038](https://github.com/ClickHouse/ClickHouse/issues/68038): Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Backported in [#68224](https://github.com/ClickHouse/ClickHouse/issues/68224): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68095](https://github.com/ClickHouse/ClickHouse/issues/68095): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68126](https://github.com/ClickHouse/ClickHouse/issues/68126): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68223](https://github.com/ClickHouse/ClickHouse/issues/68223): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68175](https://github.com/ClickHouse/ClickHouse/issues/68175): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68341](https://github.com/ClickHouse/ClickHouse/issues/68341): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68398](https://github.com/ClickHouse/ClickHouse/issues/68398): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68669](https://github.com/ClickHouse/ClickHouse/issues/68669): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67803](https://github.com/ClickHouse/ClickHouse/issues/67803): Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68081](https://github.com/ClickHouse/ClickHouse/issues/68081): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Update version after release. [#68044](https://github.com/ClickHouse/ClickHouse/pull/68044) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68269](https://github.com/ClickHouse/ClickHouse/issues/68269): [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#68432](https://github.com/ClickHouse/ClickHouse/issues/68432): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68538](https://github.com/ClickHouse/ClickHouse/issues/68538): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68555](https://github.com/ClickHouse/ClickHouse/issues/68555): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). +* Backported in [#68760](https://github.com/ClickHouse/ClickHouse/issues/68760): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 57a59d7ac49..d9674ed2366 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.4.51-stable 2024-08-23 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 From 6ba686d2510a2d95ab4332560163d0b4600533a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 23 Aug 2024 09:20:40 +0000 Subject: [PATCH 311/409] Split test case and reduce number of random runs to reduce time necessary to run the test --- .../01395_limit_more_cases.reference | 1 - .../0_stateless/01395_limit_more_cases.sh | 24 ++++--------------- .../01395_limit_more_cases_random.reference | 1 + .../01395_limit_more_cases_random.sh | 22 +++++++++++++++++ 4 files changed, 28 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/01395_limit_more_cases_random.reference create mode 100755 tests/queries/0_stateless/01395_limit_more_cases_random.sh diff --git a/tests/queries/0_stateless/01395_limit_more_cases.reference b/tests/queries/0_stateless/01395_limit_more_cases.reference index c9d0dd73ab8..d68b987ea19 100644 --- a/tests/queries/0_stateless/01395_limit_more_cases.reference +++ b/tests/queries/0_stateless/01395_limit_more_cases.reference @@ -254,4 +254,3 @@ 15 13 0 0 0 0 0 0 15 14 0 0 0 0 0 0 15 15 0 0 0 0 0 0 -0 0 0 diff --git a/tests/queries/0_stateless/01395_limit_more_cases.sh b/tests/queries/0_stateless/01395_limit_more_cases.sh index 177147d2142..9709bd74f26 100755 --- a/tests/queries/0_stateless/01395_limit_more_cases.sh +++ b/tests/queries/0_stateless/01395_limit_more_cases.sh @@ -9,8 +9,11 @@ SIZE=13 for OFFSET in {0..15}; do for LIMIT in {0..15}; do echo "SELECT - $OFFSET, $LIMIT, - count() AS c, min(number) AS first, max(number) AS last, + $OFFSET, + $LIMIT, + count() AS c, + min(number) AS first, + max(number) AS last, throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) @@ -18,20 +21,3 @@ for OFFSET in {0..15}; do " done done | $CLICKHOUSE_CLIENT -n --max_block_size 5 - -# Randomized test - -ITERATIONS=1000 -for _ in $(seq $ITERATIONS); do - SIZE=$(($RANDOM % 100)) - OFFSET=$(($RANDOM % 111)) - LIMIT=$(($RANDOM % 111)) - - echo "WITH count() AS c, min(number) AS first, max(number) AS last - SELECT - throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), - throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), - throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) - FROM (SELECT * FROM numbers($SIZE) LIMIT $OFFSET, $LIMIT); - " -done | $CLICKHOUSE_CLIENT -n --max_block_size $(($RANDOM % 20 + 1)) | uniq diff --git a/tests/queries/0_stateless/01395_limit_more_cases_random.reference b/tests/queries/0_stateless/01395_limit_more_cases_random.reference new file mode 100644 index 00000000000..06b63ea6c2f --- /dev/null +++ b/tests/queries/0_stateless/01395_limit_more_cases_random.reference @@ -0,0 +1 @@ +0 0 0 diff --git a/tests/queries/0_stateless/01395_limit_more_cases_random.sh b/tests/queries/0_stateless/01395_limit_more_cases_random.sh new file mode 100755 index 00000000000..c2f6b060aab --- /dev/null +++ b/tests/queries/0_stateless/01395_limit_more_cases_random.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SIZE=13 +ITERATIONS=300 +for _ in $(seq $ITERATIONS); do + SIZE=$(($RANDOM % 100)) + OFFSET=$(($RANDOM % 111)) + LIMIT=$(($RANDOM % 111)) + + echo "WITH count() AS c, min(number) AS first, max(number) AS last + SELECT + throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), + throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), + throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) + FROM (SELECT * FROM numbers($SIZE) LIMIT $OFFSET, $LIMIT); + " +done | $CLICKHOUSE_CLIENT -n --max_block_size $(($RANDOM % 20 + 1)) | uniq From 8c4329964f597b1eb8139990a41360243f9337f9 Mon Sep 17 00:00:00 2001 From: Maxim Dergousov Date: Fri, 23 Aug 2024 12:50:18 +0300 Subject: [PATCH 312/409] 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 313/409] 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 b0894bffe62722acee2fa5d832ceda9a75754bde Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 12:01:17 +0200 Subject: [PATCH 314/409] change test file location --- .../sample.parquet | Bin 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/{column0=Elizabeth => column0=Elizabeth1}/sample.parquet (100%) diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet rename to tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet From b1e0469d2b085ecc6ff1628a2c707a89b78bb12c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 10:58:43 +0000 Subject: [PATCH 315/409] Check setting in runtime --- src/DataTypes/DataTypeObject.cpp | 17 +++++++++++++++-- src/DataTypes/DataTypeObjectDeprecated.cpp | 4 ---- .../03230_json_alias_new_old_types.reference | 2 ++ .../03230_json_alias_new_old_types.sql | 8 ++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03230_json_alias_new_old_types.reference create mode 100644 tests/queries/0_stateless/03230_json_alias_new_old_types.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index a56764f4e6e..62239372b28 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -1,10 +1,12 @@ #include #include +#include #include #include #include #include #include +#include #include #include @@ -513,13 +515,24 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: static DataTypePtr createJSON(const ASTPtr & arguments) { + auto context = CurrentThread::getQueryContext(); + if (!context) + context = Context::getGlobalContextInstance(); + + if (context->getSettingsRef().use_json_alias_for_old_object_type) + { + if (arguments && !arguments->children.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If ypu want to use new JSON type, set setting allow_experimental_json_type = 1"); + + return std::make_shared("JSON", false); + } + return createObject(arguments, DataTypeObject::SchemaFormat::JSON); } void registerDataTypeJSON(DataTypeFactory & factory) { - if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive); + factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp index 07f9c116e58..2ef3098811d 100644 --- a/src/DataTypes/DataTypeObjectDeprecated.cpp +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -78,10 +78,6 @@ static DataTypePtr create(const ASTPtr & arguments) void registerDataTypeObjectDeprecated(DataTypeFactory & factory) { factory.registerDataType("Object", create); - if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerSimpleDataType("JSON", - [] { return std::make_shared("JSON", false); }, - DataTypeFactory::Case::Insensitive); } } diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference new file mode 100644 index 00000000000..91fd15697be --- /dev/null +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference @@ -0,0 +1,2 @@ +{"a":42} JSON +{"a":42} Object(\'json\') diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql new file mode 100644 index 00000000000..97d64f2a606 --- /dev/null +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql @@ -0,0 +1,8 @@ +set allow_experimental_object_type=1; +set allow_experimental_json_type=1; +set use_json_alias_for_old_object_type=0; +select '{"a" : 42}'::JSON as json, toTypeName(json); +set use_json_alias_for_old_object_type=1; +select '{"a" : 42}'::JSON as json, toTypeName(json); +select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS} + From 5c5c803fad08600d9ca667f2246b479e6bff5eb9 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 11:00:03 +0000 Subject: [PATCH 316/409] Update docs --- docs/en/operations/settings/settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 19db4be17db..f2e75e34609 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5623,7 +5623,6 @@ Default value: `1GiB`. ## use_json_alias_for_old_object_type When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type. -This setting requires server restart to take effect when changed. Default value: `false`. From 1165ae756d3a6ca1b9b7c7e9be77f1812390c527 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 12:16:16 +0000 Subject: [PATCH 317/409] Make dynamic structure selection more consistent --- src/Columns/ColumnDynamic.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..efb835b2e17 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1182,12 +1182,13 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source if (!canAddNewVariants(0, all_variants.size())) { /// Create list of variants with their sizes and sort it. - std::vector> variants_with_sizes; + std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) { - if (variant->getName() != getSharedVariantTypeName()) - variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + auto variant_name = variant->getName(); + if (variant_name != getSharedVariantTypeName()) + variants_with_sizes.emplace_back(total_sizes[variant_name], variant_name, variant); } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); @@ -1196,14 +1197,14 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. /// Add shared variant. result_variants.push_back(getSharedVariantDataType()); - for (const auto & [size, variant] : variants_with_sizes) + for (const auto & [size, variant_name, variant_type] : variants_with_sizes) { /// Add variant to the resulting variants list until we reach max_dynamic_types. if (canAddNewVariant(result_variants.size())) - result_variants.push_back(variant); + result_variants.push_back(variant_type); /// Add all remaining variants into shared_variants_statistics until we reach its max size. else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) - new_statistics.shared_variants_statistics[variant->getName()] = size; + new_statistics.shared_variants_statistics[variant_name] = size; else break; } From ef7970d2c38a301530721751e7bbde908c84721a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 14:19:42 +0200 Subject: [PATCH 318/409] Fix typo --- src/DataTypes/DataTypeObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 62239372b28..bb0bb928b4f 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -522,7 +522,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments) if (context->getSettingsRef().use_json_alias_for_old_object_type) { if (arguments && !arguments->children.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If ypu want to use new JSON type, set setting allow_experimental_json_type = 1"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set setting allow_experimental_json_type = 1"); return std::make_shared("JSON", false); } From 6f5210644b95b41cc9d490d4e117c81bd61a1d06 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 14:43:09 +0200 Subject: [PATCH 319/409] Update src/Columns/ColumnObject.cpp Co-authored-by: Alexander Gololobov --- src/Columns/ColumnObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 999c0f6088e..e397b03b69e 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1045,7 +1045,7 @@ void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColu bool ColumnObject::structureEquals(const IColumn & rhs) const { - /// 2 Object columns have equal structure if they have the same typed paths and max_dynamic_paths/max_dynamic_types. + /// 2 Object columns have equal structure if they have the same typed paths and global_max_dynamic_paths/max_dynamic_types. const auto * rhs_object = typeid_cast(&rhs); if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || global_max_dynamic_paths != rhs_object->global_max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types) return false; From 2b20b2d4de78acf4fbb08b3f106ebdf410e4587d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 15:02:43 +0200 Subject: [PATCH 320/409] Update src/Columns/ColumnDynamic.cpp Co-authored-by: Dmitry Novik --- src/Columns/ColumnDynamic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index efb835b2e17..ef6cd7dcea2 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1181,7 +1181,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// Check if the number of all dynamic types exceeds the limit. if (!canAddNewVariants(0, all_variants.size())) { - /// Create list of variants with their sizes and sort it. + /// Create a list of variants with their sizes and names and then sort it. std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) From 5d6b861ff055de0d04e0c574bf2ebb1e51215ace Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 13:49:36 +0000 Subject: [PATCH 321/409] Fix index with limit=0 --- src/Columns/ColumnVariant.cpp | 2 +- .../03228_variant_permutation_issue.reference | 4 ++++ .../03228_variant_permutation_issue.sql | 15 +++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 2fea3eca123..c6511695f5c 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -953,7 +953,7 @@ ColumnPtr ColumnVariant::index(const IColumn & indexes, size_t limit) const { /// If we have only NULLs, index will take no effect, just return resized column. if (hasOnlyNulls()) - return cloneResized(limit); + return cloneResized(limit == 0 ? indexes.size(): limit); /// Optimization when we have only one non empty variant and no NULLs. /// In this case local_discriminators column is filled with identical values and offsets column diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.reference b/tests/queries/0_stateless/03228_variant_permutation_issue.reference index 10688253e15..be9cdedaf07 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.reference +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.reference @@ -2,3 +2,7 @@ 3 {"foo2":"bar"} 1 2 {"foo2":"baz"} 2 3 {"foo2":"bar"} 1 +2 {"foo2":"bar"} 1 +3 {"foo2":"bar"} 1 +2 {"foo2":"baz"} 2 +3 {"foo2":"bar"} 1 diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.sql b/tests/queries/0_stateless/03228_variant_permutation_issue.sql index 088361d6430..81eb2ed69af 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.sql +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.sql @@ -16,3 +16,18 @@ SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; DROP TABLE test_new_json_type; +CREATE TABLE test_new_json_type(id Nullable(UInt32), data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id settings allow_nullable_key=1; +INSERT INTO test_new_json_type format JSONEachRow +{"id":1,"data":{"foo1":"bar"},"version":1} +{"id":2,"data":{"foo2":"bar"},"version":1} +{"id":3,"data":{"foo2":"bar"},"version":1} +; + +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; + +INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2; + +SELECT * FROM test_new_json_type FINAL PREWHERE data.foo2 IS NOT NULL WHERE data.foo2 IS NOT NULL ORDER BY id ASC NULLS FIRST; + +DROP TABLE test_new_json_type; + From eb25e064acf0684cea2b0503f78659431a7c57b4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 13:53:41 +0000 Subject: [PATCH 322/409] reduce amount --- .../02477_projection_materialize_and_zero_copy.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 2750bdf5c4b..0597ac10cd7 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -3,9 +3,9 @@ DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 -SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8092, index_granularity_bytes='10Mi'; +SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8192, index_granularity_bytes='10Mi'; -insert into t (c1, c18) select number, -number from numbers(2000000); +insert into t (c1, c18) select number, -number from numbers(500000); alter table t add projection p_norm (select * order by c1); From 61fa4e7a476b3db31c22030470341b131501f3b6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 16:38:48 +0200 Subject: [PATCH 323/409] fix logical err of modify statistics --- src/Storages/AlterCommands.cpp | 2 +- src/Storages/StatisticsDescription.cpp | 6 ++++-- .../integration/test_manipulate_statistics/test.py | 14 ++++++++++++-- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d5780e32db3..67b18217767 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -734,7 +734,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { if (!metadata.columns.has(statistics_column_name)) { - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot modify statistics for column {}: this column is not found", statistics_column_name); } } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 63c849e3806..acf600dd6f7 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include @@ -115,8 +114,11 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other) { + /// If the statistics is empty, it's possible that we have not assign a column_name. + if (empty() && column_name == "") + column_name = other.column_name; if (other.column_name != column_name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", other.column_name, column_name); types_to_desc = other.types_to_desc; data_type = other.data_type; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 2541c9b946f..ab5559e18fa 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,11 +6,13 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=True + "node1", user_configs=["config/config.xml"], with_zookeeper=True, + macros={"replica": "a", "shard": "shard1"} ) node2 = cluster.add_instance( - "node2", user_configs=["config/config.xml"], with_zookeeper=True + "node2", user_configs=["config/config.xml"], with_zookeeper=True, + macros={"replica": "b", "shard": "shard1"} ) @@ -183,3 +185,11 @@ def test_replicated_table_ddl(started_cluster): ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) + + +def test_replicated_db(started_cluster): + node1.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") + node2.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") + node1.query("CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()") + node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64") + node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest") From 9ad7cfc71ff6e61a51622740b4aff72857389ecd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 23 Aug 2024 23:00:42 +0800 Subject: [PATCH 324/409] host fix lower upper performance issue --- src/Functions/LowerUpperUTF8Impl.h | 74 ++++++++++++++++++++++-------- 1 file changed, 54 insertions(+), 20 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5da085f48e5..490b53b0f4a 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -6,8 +6,10 @@ #include #include -#include #include +#include +#include +#include #include namespace DB @@ -38,38 +40,70 @@ struct LowerUpperUTF8Impl return; } + + UErrorCode error_code = U_ZERO_ERROR; + UCaseMap * csm = ucasemap_open(nullptr, 0, &error_code); + if (U_FAILURE(error_code)) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Call ucasemap_open error:{}", u_errorName(error_code)); + + // String output; + size_t curr_offset = 0; res_data.resize(data.size()); res_offsets.resize_exact(offsets.size()); - - String output; - size_t curr_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); - size_t size = offsets[i] - offsets[i - 1]; + size_t size = offsets[i] - offsets[i - 1] - 1; - icu::UnicodeString input(data_start, static_cast(size), "UTF-8"); + int32_t out_size; if constexpr (upper) - input.toUpper(); + out_size = ucasemap_utf8ToUpper( + csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); else - input.toLower(); + out_size = ucasemap_utf8ToLower( + csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code + // << std::endl; - output.clear(); - input.toUTF8String(output); + if (error_code == U_BUFFER_OVERFLOW_ERROR) + { + size_t new_size = curr_offset + out_size + 1; + res_data.resize(new_size); - /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first - /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this - /// case, the behavior is also reasonable. - const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); - size_t valid_size = res_end ? res_end - output.data() + 1 : 0; + error_code = U_ZERO_ERROR; + if constexpr (upper) + out_size = ucasemap_utf8ToUpper( + csm, + reinterpret_cast(&res_data[curr_offset]), + res_data.size() - curr_offset, + data_start, + size, + &error_code); + else + out_size = ucasemap_utf8ToLower( + csm, + reinterpret_cast(&res_data[curr_offset]), + res_data.size() - curr_offset, + data_start, + size, + &error_code); + } - res_data.resize(curr_offset + valid_size + 1); - memcpy(&res_data[curr_offset], output.data(), valid_size); - res_data[curr_offset + valid_size] = 0; + if (error_code != U_ZERO_ERROR) + throw DB::Exception( + ErrorCodes::LOGICAL_ERROR, + "Call {} error:{} input:{} input_size:{}", + upper ? "ucasemap_utf8ToUpper" : "ucasemap_utf8ToLower", + u_errorName(error_code), + std::string_view(data_start, size), + size); - curr_offset += valid_size + 1; + res_data[curr_offset + out_size] = 0; + curr_offset += out_size + 1; res_offsets[i] = curr_offset; } + + res_data.resize(curr_offset); } static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) From 223de640721a84b1434417379bd47636ae3b8841 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 17:09:28 +0200 Subject: [PATCH 325/409] init --- .../__init__.py | 0 .../configs/config.d/cluster.xml | 11 ---- .../configs/config.xml | 9 ---- .../test_incorrect_datetime_format/test.py | 54 ------------------- ...301_test_incorrect_datetime_msan.reference | 3 ++ .../032301_test_incorrect_datetime_msan.sh | 20 +++++++ 6 files changed, 23 insertions(+), 74 deletions(-) delete mode 100644 tests/integration/test_incorrect_datetime_format/__init__.py delete mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml delete mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.xml delete mode 100644 tests/integration/test_incorrect_datetime_format/test.py create mode 100644 tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference create mode 100644 tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh diff --git a/tests/integration/test_incorrect_datetime_format/__init__.py b/tests/integration/test_incorrect_datetime_format/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml deleted file mode 100644 index a27968fb3d2..00000000000 --- a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - node - - - - - diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.xml b/tests/integration/test_incorrect_datetime_format/configs/config.xml deleted file mode 100644 index 053b5d30418..00000000000 --- a/tests/integration/test_incorrect_datetime_format/configs/config.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - information - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py deleted file mode 100644 index 3cdc6781534..00000000000 --- a/tests/integration/test_incorrect_datetime_format/test.py +++ /dev/null @@ -1,54 +0,0 @@ -import logging -import pytest -from helpers.cluster import ClickHouseCluster - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node", - main_configs=[ - "configs/config.d/cluster.xml", - ], - ) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") - - node = cluster.instances["node"] - node.query( - """ - CREATE TABLE tab - ( - a DateTime, - pk String - ) Engine = MergeTree() ORDER BY pk; - """ - ) - - yield cluster - finally: - cluster.shutdown() - - -def test_incorrect_datetime_format(cluster): - """ - Test for an MSan issue which is caused by parsing incorrect datetime string - """ - - node = cluster.instances["node"] - - res = node.query("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'").strip() - assert res == "0" - - error = node.query_and_get_error( - "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'" - ).strip() - assert "Cannot parse time component of DateTime 09:58:0" in error - - error = node.query_and_get_error( - "SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'" - ).strip() - assert "Cannot convert string '2024-08-0 09:58:09' to type DateTime" in error diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference new file mode 100644 index 00000000000..346e6731867 --- /dev/null +++ b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference @@ -0,0 +1,3 @@ +0 +OK +OK diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh new file mode 100644 index 00000000000..c54c69dce58 --- /dev/null +++ b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " +CREATE TABLE tab +( + a DateTime, + pk String +) Engine = MergeTree() ORDER BY pk; +" + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'" + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'" 2>&1 | grep -F -q "Cannot parse time component of DateTime 09:58:0" && echo "OK" || echo "FAIL"; + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'" 2>&1 | grep -F -q "Cannot convert string '2024-08-0 09:58:09" && echo "OK" || echo "FAIL"; From 7aabd7d2fd4a03ddea5ef311cf89b2eb7520674c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 15:11:51 +0000 Subject: [PATCH 326/409] Fix resolving dynamic subcolumns from subqueries in analyzer --- src/Analyzer/Resolve/IdentifierResolver.cpp | 2 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 3 +++ src/Analyzer/Resolve/TableExpressionData.h | 1 + .../03228_dynamic_subcolumns_from_subquery.reference | 4 ++++ .../03228_dynamic_subcolumns_from_subquery.sql | 9 +++++++++ 5 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference create mode 100644 tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 14d4acc7c9b..80e7d1e4445 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -692,7 +692,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( result_column_node = it->second; } /// Check if it's a dynamic subcolumn - else + else if (table_expression_data.supports_subcolumns) { auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name); auto jt = table_expression_data.column_name_to_column_node.find(column_name); diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 004da5ed341..a18c2901a58 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4379,7 +4379,10 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); if (storage_snapshot->storage.supportsSubcolumns()) + { get_column_options.withSubcolumns(); + table_expression_data.supports_subcolumns = true; + } auto column_names_and_types = storage_snapshot->getColumns(get_column_options); table_expression_data.column_names_and_types = NamesAndTypes(column_names_and_types.begin(), column_names_and_types.end()); diff --git a/src/Analyzer/Resolve/TableExpressionData.h b/src/Analyzer/Resolve/TableExpressionData.h index 18cbfa32366..6770672d0c2 100644 --- a/src/Analyzer/Resolve/TableExpressionData.h +++ b/src/Analyzer/Resolve/TableExpressionData.h @@ -36,6 +36,7 @@ struct AnalysisTableExpressionData std::string database_name; std::string table_name; bool should_qualify_columns = true; + bool supports_subcolumns = false; NamesAndTypes column_names_and_types; ColumnNameToColumnNodeMap column_name_to_column_node; std::unordered_set subcolumn_names; /// Subset columns that are subcolumns of other columns diff --git a/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference new file mode 100644 index 00000000000..153ad78f694 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference @@ -0,0 +1,4 @@ +str +42 +42 +42 diff --git a/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql new file mode 100644 index 00000000000..a10b0cb2809 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql @@ -0,0 +1,9 @@ +set allow_experimental_dynamic_type=1; +set allow_experimental_json_type=1; +set allow_experimental_analyzer=1; + +select d.String from (select 'str'::Dynamic as d); +select json.a from (select '{"a" : 42}'::JSON as json); +select json.a from (select '{"a" : 42}'::JSON(a UInt32) as json); +select json.a.:Int64 from (select materialize('{"a" : 42}')::JSON as json); + From 327828777999447756823fe1e6c99cb867896154 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 15:26:45 +0000 Subject: [PATCH 327/409] 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 328/409] 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 b83c6239acc14d1c8f1644abe6b25debb95cc8fd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 18:21:06 +0200 Subject: [PATCH 329/409] rename tests --- ...san.reference => 03231_test_incorrect_datetime_msan.reference} | 0 ...ect_datetime_msan.sh => 03231_test_incorrect_datetime_msan.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{032301_test_incorrect_datetime_msan.reference => 03231_test_incorrect_datetime_msan.reference} (100%) rename tests/queries/0_stateless/{032301_test_incorrect_datetime_msan.sh => 03231_test_incorrect_datetime_msan.sh} (100%) diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.reference similarity index 100% rename from tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference rename to tests/queries/0_stateless/03231_test_incorrect_datetime_msan.reference diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh similarity index 100% rename from tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh rename to tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh From bea98fed5e2107e621dc3f7578ee3eed14e2c5b5 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 16:45:38 +0000 Subject: [PATCH 330/409] chmod +x on test --- tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh diff --git a/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh old mode 100644 new mode 100755 From 80504e7b9b52fec79a89e2fff5881ca397022107 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 19:07:25 +0000 Subject: [PATCH 331/409] fix test 03228_virtual_column_merge_dist --- .../queries/0_stateless/03228_virtual_column_merge_dist.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql index caf00a2e407..e58c7f38d3b 100644 --- a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql @@ -1,3 +1,6 @@ +-- There is a bug in old analyzer with currentDatabase() and distributed query. +SET enable_analyzer = 1; + DROP TABLE IF EXISTS t_local_1; DROP TABLE IF EXISTS t_local_2; DROP TABLE IF EXISTS t_merge; @@ -10,7 +13,7 @@ INSERT INTO t_local_1 VALUES (1); INSERT INTO t_local_2 VALUES (2); CREATE TABLE t_merge AS t_local_1 ENGINE = Merge(currentDatabase(), '^(t_local_1|t_local_2)$'); -CREATE TABLE t_distr AS t_local_1 engine=Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); +CREATE TABLE t_distr AS t_local_1 ENGINE = Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); SELECT a, _table FROM t_merge ORDER BY a; SELECT a, _table FROM t_distr ORDER BY a; From 7916490afa8d4ac54bcd8b541dfe02ec9a715d3a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 20:26:31 +0000 Subject: [PATCH 332/409] add more cases for test --- .../Executors/StreamingFormatExecutor.cpp | 10 ++-------- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- ...ync_insert_query_params_bad_type.reference | 1 + ...228_async_insert_query_params_bad_type.sql | 20 +++++++++++++++++++ 4 files changed, 24 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 5b35ac6020d..10a7b7fd7f5 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -35,15 +35,9 @@ MutableColumns StreamingFormatExecutor::getResultColumns() void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters) { - if (parameters.empty()) - return; - /// Query parameters make sense only for format Values. - auto * values_format = typeid_cast(format.get()); - if (!values_format) - return; - - values_format->setQueryParameters(parameters); + if (auto * values_format = typeid_cast(format.get())) + values_format->setQueryParameters(parameters); } size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index e34000df5f4..10d0e051665 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -665,7 +665,7 @@ void ValuesBlockInputFormat::resetReadBuffer() void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters) { - if (parameters.empty()) + if (parameters == context->getQueryParameters()) return; auto context_copy = Context::createCopy(context); diff --git a/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference new file mode 100644 index 00000000000..e965047ad7c --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference @@ -0,0 +1 @@ +Hello diff --git a/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql new file mode 100644 index 00000000000..359174245cb --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t_async_insert_params; + +CREATE TABLE t_async_insert_params (id UInt64) ENGINE = MergeTree ORDER BY tuple(); + +SET param_p1 = 'Hello'; + +SET async_insert = 1; +SET wait_for_async_insert = 1; + +INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER } +INSERT INTO t_async_insert_params VALUES ({p1:String}); -- { serverError TYPE_MISMATCH } + +ALTER TABLE t_async_insert_params MODIFY COLUMN id String; + +INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER } +INSERT INTO t_async_insert_params VALUES ({p1:String}); + +SELECT * FROM t_async_insert_params ORDER BY id; + +DROP TABLE t_async_insert_params; From a82421719383041a839289093d1882265a068cd1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 20:29:04 +0000 Subject: [PATCH 333/409] Done --- ..._rewrite_sum_column_and_constant.reference | 26 +++++++++---------- ...alyzer_rewrite_sum_column_and_constant.sql | 11 ++++---- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference index 802d920aaef..b41635f014e 100644 --- a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference +++ b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference @@ -1635,21 +1635,21 @@ QUERY id: 0 JOIN TREE TABLE id: 10, alias: __table1, table_name: default.test_table SELECT sum(float64 + 2) From test_table; -26.5 +26.875 SELECT sum(2 + float64) From test_table; -26.5 +26.875 SELECT sum(float64 - 2) From test_table; -6.5 +6.875 SELECT sum(2 - float64) From test_table; --6.5 +-6.875 SELECT sum(float64) + 2 * count(float64) From test_table; -26.5 +26.875 SELECT 2 * count(float64) + sum(float64) From test_table; -26.5 +26.875 SELECT sum(float64) - 2 * count(float64) From test_table; -6.5 +6.875 SELECT 2 * count(float64) - sum(float64) From test_table; --6.5 +-6.875 EXPLAIN QUERY TREE (SELECT sum(float64 + 2) From test_table); QUERY id: 0 PROJECTION COLUMNS @@ -2463,25 +2463,25 @@ QUERY id: 0 JOIN TREE TABLE id: 12, alias: __table1, table_name: default.test_table SELECT sum(float64 + 2) + sum(float64 + 3) From test_table; -58 +58.75 SELECT sum(float64 + 2) - sum(float64 + 3) From test_table; -5 SELECT sum(float64 - 2) + sum(float64 - 3) From test_table; -8 +8.75 SELECT sum(float64 - 2) - sum(float64 - 3) From test_table; 5 SELECT sum(2 - float64) - sum(3 - float64) From test_table; -5 SELECT (sum(float64) + 2 * count(float64)) + (sum(float64) + 3 * count(float64)) From test_table; -58 +58.75 SELECT (sum(float64) + 2 * count(float64)) - (sum(float64) + 3 * count(float64)) From test_table; -5 SELECT (sum(float64) - 2 * count(float64)) + (sum(float64) - 3 * count(float64)) From test_table; -8 +8.75 SELECT (sum(float64) - 2 * count(float64)) - (sum(float64) - 3 * count(float64)) From test_table; 5 SELECT (2 * count(float64) - sum(float64)) + (3 * count(float64) - sum(float64)) From test_table; --8 +-8.75 EXPLAIN QUERY TREE (SELECT sum(float64 + 2) + sum(float64 + 3) From test_table); QUERY id: 0 PROJECTION COLUMNS diff --git a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql index 5492d061c12..b6fa097abe9 100644 --- a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql +++ b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql @@ -25,11 +25,12 @@ CREATE TABLE test_table decimal32 Decimal32(5), ) ENGINE=MergeTree ORDER BY uint64; -INSERT INTO test_table VALUES (1, 1.1, 1.11); -INSERT INTO test_table VALUES (2, 2.2, 2.22); -INSERT INTO test_table VALUES (3, 3.3, 3.33); -INSERT INTO test_table VALUES (4, 4.4, 4.44); -INSERT INTO test_table VALUES (5, 5.5, 5.55); +-- Use Float64 numbers divisible by 1/16 (or some other small power of two), so that their sum doesn't depend on summation order. +INSERT INTO test_table VALUES (1, 1.125, 1.11); +INSERT INTO test_table VALUES (2, 2.250, 2.22); +INSERT INTO test_table VALUES (3, 3.375, 3.33); +INSERT INTO test_table VALUES (4, 4.500, 4.44); +INSERT INTO test_table VALUES (5, 5.625, 5.55); -- { echoOn } SELECT sum(uint64 + 1 AS i) from test_table where i > 0; From 0f265ce33d857a9c7446698629b6517b71b4a71d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 23:13:53 +0200 Subject: [PATCH 334/409] address comments --- src/Interpreters/InterpreterCreateQuery.cpp | 1 - src/Storages/AlterCommands.cpp | 10 ++++------ src/Storages/ColumnsDescription.cpp | 4 ---- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 ++-- src/Storages/Statistics/Statistics.cpp | 16 ++++++++-------- src/Storages/Statistics/Statistics.h | 6 ++++-- src/Storages/StatisticsDescription.cpp | 21 +++++---------------- src/Storages/StatisticsDescription.h | 4 ++-- 9 files changed, 26 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 95143031707..467547e6c9e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -700,7 +700,6 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); } - column.statistics.column_name = column.name; /// We assign column name here for better exception error message. if (col_decl.statistics_desc) { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 67b18217767..07bc87b0162 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -705,9 +705,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns); - for (const auto & stats : stats_vec) + for (const auto & [stats_column_name, stats] : stats_vec) { - metadata.columns.modify(stats.column_name, + metadata.columns.modify(stats_column_name, [&](ColumnDescription & column) { column.statistics.merge(stats, column.name, column.type, if_not_exists); }); } } @@ -739,9 +739,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns); - for (const auto & stats : stats_vec) + for (const auto & [stats_column_name, stats] : stats_vec) { - metadata.columns.modify(stats.column_name, + metadata.columns.modify(stats_column_name, [&](ColumnDescription & column) { column.statistics.assign(stats); }); } } @@ -866,8 +866,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) rename_visitor.visit(column_to_modify.default_desc.expression); if (column_to_modify.ttl) rename_visitor.visit(column_to_modify.ttl); - if (column_to_modify.name == column_name && !column_to_modify.statistics.empty()) - column_to_modify.statistics.column_name = rename_to; }); } if (metadata.table_ttl.definition_ast) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d724245b49..fdc3446aa46 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -209,11 +209,7 @@ void ColumnDescription::readText(ReadBuffer & buf) settings = col_ast->settings->as().changes; if (col_ast->statistics_desc) - { statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type); - /// every column has name `x` here, so we have to set the name manually. - statistics.column_name = name; - } } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ce06adf110c..0d34eb7f630 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -62,7 +62,7 @@ static ColumnsStatistics getStatisticsForColumns( const auto * desc = all_columns.tryGet(column.name); if (desc && !desc->statistics.empty()) { - auto statistics = MergeTreeStatisticsFactory::instance().get(desc->statistics); + auto statistics = MergeTreeStatisticsFactory::instance().get(*desc); all_statistics.push_back(std::move(statistics)); } } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b9b5333a61c..1119ca324d6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -546,7 +546,7 @@ static std::set getStatisticsToRecalculate(const StorageMet { if (!col_desc.statistics.empty() && materialized_stats.contains(col_desc.name)) { - stats_to_recalc.insert(stats_factory.get(col_desc.statistics)); + stats_to_recalc.insert(stats_factory.get(col_desc)); } } return stats_to_recalc; @@ -1530,7 +1530,7 @@ private: if (ctx->materialized_statistics.contains(col.name)) { - stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col.statistics)); + stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col)); } else { diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index fd686c5f0aa..6372c804e0e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -58,8 +58,8 @@ IStatistics::IStatistics(const SingleStatisticsDescription & stat_) { } -ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) - : stats_desc(stats_desc_) +ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_) + : stats_desc(stats_desc_), column_name(column_name_) { } @@ -176,7 +176,7 @@ String ColumnStatistics::getFileName() const const String & ColumnStatistics::columnName() const { - return stats_desc.column_name; + return column_name; } UInt64 ColumnStatistics::rowCount() const @@ -227,15 +227,15 @@ void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & st } } -ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const +ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnDescription & column_desc) const { - ColumnStatisticsPtr column_stat = std::make_shared(stats); - for (const auto & [type, desc] : stats.types_to_desc) + ColumnStatisticsPtr column_stat = std::make_shared(column_desc.statistics, column_desc.name); + for (const auto & [type, desc] : column_desc.statistics.types_to_desc) { auto it = creators.find(type); if (it == creators.end()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type); - auto stat_ptr = (it->second)(desc, stats.data_type); + auto stat_ptr = (it->second)(desc, column_desc.type); column_stat->stats[type] = stat_ptr; } return column_stat; @@ -246,7 +246,7 @@ ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & ColumnsStatistics result; for (const auto & col : columns) if (!col.statistics.empty()) - result.push_back(get(col.statistics)); + result.push_back(get(col)); return result; } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 2a30c0de315..98666ed73df 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -54,7 +54,7 @@ using StatisticsPtr = std::shared_ptr; class ColumnStatistics { public: - explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_); + explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_); void serialize(WriteBuffer & buf); void deserialize(ReadBuffer & buf); @@ -73,10 +73,12 @@ public: private: friend class MergeTreeStatisticsFactory; ColumnStatisticsDescription stats_desc; + String column_name; std::map stats; UInt64 rows = 0; /// the number of rows in the column }; +struct ColumnDescription; class ColumnsDescription; using ColumnStatisticsPtr = std::shared_ptr; using ColumnsStatistics = std::vector; @@ -91,7 +93,7 @@ public: using Validator = std::function; using Creator = std::function; - ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; + ColumnStatisticsPtr get(const ColumnDescription & column_desc) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const; void registerValidator(StatisticsType type, Validator validator); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index acf600dd6f7..64634124758 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -96,16 +96,13 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe { chassert(merging_column_type); - if (column_name.empty()) - column_name = merging_column_name; - data_type = merging_column_type; for (const auto & [stats_type, stats_desc]: other.types_to_desc) { if (!if_not_exists && types_to_desc.contains(stats_type)) { - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, merging_column_name); } else if (!types_to_desc.contains(stats_type)) types_to_desc.emplace(stats_type, stats_desc); @@ -114,12 +111,6 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other) { - /// If the statistics is empty, it's possible that we have not assign a column_name. - if (empty() && column_name == "") - column_name = other.column_name; - if (other.column_name != column_name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", other.column_name, column_name); - types_to_desc = other.types_to_desc; data_type = other.data_type; } @@ -129,7 +120,7 @@ void ColumnStatisticsDescription::clear() types_to_desc.clear(); } -std::vector ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) +std::vector> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) { const auto * stat_definition_ast = definition_ast->as(); if (!stat_definition_ast) @@ -147,7 +138,7 @@ std::vector ColumnStatisticsDescription::fromAST(co statistics_types.emplace(stat.type, stat); } - std::vector result; + std::vector> result; result.reserve(stat_definition_ast->columns->children.size()); for (const auto & column_ast : stat_definition_ast->columns->children) @@ -159,10 +150,9 @@ std::vector ColumnStatisticsDescription::fromAST(co throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name); const auto & column = columns.getPhysical(physical_column_name); - stats.column_name = column.name; stats.data_type = column.type; stats.types_to_desc = statistics_types; - result.push_back(stats); + result.emplace_back(physical_column_name, stats); } if (result.empty()) @@ -177,14 +167,13 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c if (stat_type_list_ast->children.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); ColumnStatisticsDescription stats; - stats.column_name = column.name; for (const auto & ast : stat_type_list_ast->children) { const auto & stat_type = ast->as().name; SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone()); if (stats.types_to_desc.contains(stat.type)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", column.name, stat_type); stats.types_to_desc.emplace(stat.type, std::move(stat)); } stats.data_type = data_type; diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 03b8fb0d583..46927f1418c 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -55,12 +55,12 @@ struct ColumnStatisticsDescription ASTPtr getAST() const; - static std::vector fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); + /// get a vector of pair + static std::vector> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type); using StatisticsTypeDescMap = std::map; StatisticsTypeDescMap types_to_desc; - String column_name; DataTypePtr data_type; }; From 6fb8f2b4ee10a95104bf6f8880471d24d39095dc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 23:19:03 +0200 Subject: [PATCH 335/409] fix black --- .../test_manipulate_statistics/test.py | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index ab5559e18fa..aff943e4d20 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,13 +6,17 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=True, - macros={"replica": "a", "shard": "shard1"} + "node1", + user_configs=["config/config.xml"], + with_zookeeper=True, + macros={"replica": "a", "shard": "shard1"}, ) node2 = cluster.add_instance( - "node2", user_configs=["config/config.xml"], with_zookeeper=True, - macros={"replica": "b", "shard": "shard1"} + "node2", + user_configs=["config/config.xml"], + with_zookeeper=True, + macros={"replica": "b", "shard": "shard1"}, ) @@ -188,8 +192,14 @@ def test_replicated_table_ddl(started_cluster): def test_replicated_db(started_cluster): - node1.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") - node2.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") - node1.query("CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()") + node1.query( + "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" + ) + node2.query( + "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" + ) + node1.query( + "CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()" + ) node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64") node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest") From 0a35b111ffb34f3d6a8a9e9bfa712b57b722c447 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 20:03:38 +0000 Subject: [PATCH 336/409] fix test 03221_mutation_analyzer_skip_part --- .../03221_mutation_analyzer_skip_part.sh | 46 +++++++++++++++++++ .../03221_mutation_analyzer_skip_part.sql | 21 --------- 2 files changed, 46 insertions(+), 21 deletions(-) create mode 100755 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh delete mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh new file mode 100755 index 00000000000..03fd15f54e2 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-random-merge-tree-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " + DROP TABLE IF EXISTS t_mutate_skip_part; + + CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) + ENGINE = MergeTree ORDER BY id PARTITION BY key + SETTINGS min_bytes_for_wide_part = 0; + + INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); + INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); + + SET mutations_sync = 2; + ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; + ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; +" + +# Mutation query may return before the entry is added to part log. +# So, we may have to retry the flush of logs until all entries are actually flushed. +for _ in {1..10}; do + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + res=$(${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.part_log WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart'") + + if [[ $res -eq 4 ]]; then + break + fi + + sleep 2.0 +done + +${CLICKHOUSE_CLIENT} --query " + SYSTEM FLUSH LOGS; + + -- If part is skipped in mutation and hardlinked then read_rows must be 0. + SELECT part_name, read_rows + FROM system.part_log + WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' + ORDER BY part_name; + + DROP TABLE IF EXISTS t_mutate_skip_part; +" diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql deleted file mode 100644 index bf9a10e2af4..00000000000 --- a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE IF EXISTS t_mutate_skip_part; - -CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) ENGINE = MergeTree ORDER BY id PARTITION BY key; - -INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); -INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); - -SET mutations_sync = 2; - -ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; -ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; - -SYSTEM FLUSH LOGS; - --- If part is skipped in mutation and hardlinked then read_rows must be 0. -SELECT part_name, read_rows -FROM system.part_log -WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' -ORDER BY part_name; - -DROP TABLE IF EXISTS t_mutate_skip_part; From 080b8f74be186738813ca9d9e12ed3e327129c33 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Fri, 23 Aug 2024 15:50:56 -0700 Subject: [PATCH 337/409] Update README.md Add Austin meetup --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 546f08afd3d..ba212852ea8 100644 --- a/README.md +++ b/README.md @@ -54,6 +54,7 @@ Other upcoming meetups * [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 * [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 * [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 * [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 ## Recent Recordings From 5fe151529ab58112f8fa8491d2bfff24562ff624 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 24 Aug 2024 07:33:18 +0200 Subject: [PATCH 338/409] fix flacky although that is not actually flacky --- tests/integration/test_manipulate_statistics/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index aff943e4d20..3a1c5ad5b96 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -135,8 +135,8 @@ def test_single_node_normal(started_cluster): def test_replicated_table_ddl(started_cluster): - node1.query("DROP TABLE IF EXISTS test_stat") - node2.query("DROP TABLE IF EXISTS test_stat") + node1.query("DROP TABLE IF EXISTS test_stat SYNC") + node2.query("DROP TABLE IF EXISTS test_stat SYNC") node1.query( """ @@ -192,6 +192,8 @@ def test_replicated_table_ddl(started_cluster): def test_replicated_db(started_cluster): + node1.query("DROP DATABASE IF EXISTS test SYNC") + node2.query("DROP DATABASE IF EXISTS test SYNC") node1.query( "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" ) From e2aa953e700bfbabbfe69a5749f4d2806bd3610f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 24 Aug 2024 20:45:10 +0800 Subject: [PATCH 339/409] Fix empty tuple in array --- src/Functions/array/arrayElement.cpp | 3 +++ tests/queries/0_stateless/03229_empty_tuple_in_array.reference | 1 + tests/queries/0_stateless/03229_empty_tuple_in_array.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03229_empty_tuple_in_array.reference create mode 100644 tests/queries/0_stateless/03229_empty_tuple_in_array.sql diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 81f3f97979b..d0b2b49cc1c 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1598,6 +1598,9 @@ ColumnPtr FunctionArrayElement::executeTuple(const ColumnsWithTypeAndName & argu const auto & tuple_columns = col_nested->getColumns(); size_t tuple_size = tuple_columns.size(); + if (tuple_size == 0) + return ColumnTuple::create(input_rows_count); + const DataTypes & tuple_types = typeid_cast( *typeid_cast(*arguments[0].type).getNestedType()).getElements(); diff --git a/tests/queries/0_stateless/03229_empty_tuple_in_array.reference b/tests/queries/0_stateless/03229_empty_tuple_in_array.reference new file mode 100644 index 00000000000..6a452c185a8 --- /dev/null +++ b/tests/queries/0_stateless/03229_empty_tuple_in_array.reference @@ -0,0 +1 @@ +() diff --git a/tests/queries/0_stateless/03229_empty_tuple_in_array.sql b/tests/queries/0_stateless/03229_empty_tuple_in_array.sql new file mode 100644 index 00000000000..09ba3595a5a --- /dev/null +++ b/tests/queries/0_stateless/03229_empty_tuple_in_array.sql @@ -0,0 +1 @@ +select [()][0]; From 78c175225b9b4c929ed918e718351c18a166458a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 22 Aug 2024 14:50:10 +0000 Subject: [PATCH 340/409] Done --- contrib/replxx | 2 +- src/Client/ReplxxLineReader.cpp | 13 +++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/contrib/replxx b/contrib/replxx index 5d04501f93a..5f696c6eb9a 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 5d04501f93a4fb7f0bb8b73b8f614bc986f9e25b +Subproject commit 5f696c6eb9a88eb9784e8ff1d68bd5f70285dcc5 diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 78ae6c5eb15..37ceb471e5b 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -299,13 +299,14 @@ ReplxxLineReader::ReplxxLineReader( Patterns delimiters_, const char word_break_characters_[], replxx::Replxx::highlighter_callback_t highlighter_, - [[ maybe_unused ]] std::istream & input_stream_, - [[ maybe_unused ]] std::ostream & output_stream_, - [[ maybe_unused ]] int in_fd_, - [[ maybe_unused ]] int out_fd_, - [[ maybe_unused ]] int err_fd_ + std::istream & input_stream_, + std::ostream & output_stream_, + int in_fd_, + int out_fd_, + int err_fd_ ) : LineReader(history_file_path_, multiline_, std::move(extenders_), std::move(delimiters_), input_stream_, output_stream_, in_fd_) + , rx(input_stream_, output_stream_, in_fd_, out_fd_, err_fd_) , highlighter(std::move(highlighter_)) , word_break_characters(word_break_characters_) , editor(getEditor()) @@ -516,7 +517,7 @@ void ReplxxLineReader::addToHistory(const String & line) rx.history_add(line); // flush changes to the disk - if (!rx.history_save(history_file_path)) + if (history_file_fd >= 0 && !rx.history_save(history_file_path)) rx.print("Saving history failed: %s\n", errnoToString().c_str()); if (history_file_fd >= 0 && locked && 0 != flock(history_file_fd, LOCK_UN)) From 01523cce2a4ba21c9855ab4eb1398986cf66c64b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 12:14:40 +0000 Subject: [PATCH 341/409] Bump replxx --- contrib/replxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/replxx b/contrib/replxx index 5f696c6eb9a..711c18e7f4d 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 5f696c6eb9a88eb9784e8ff1d68bd5f70285dcc5 +Subproject commit 711c18e7f4d951255aa8b0851e5a55d5a5fb0ddb From 385c8127cf4b7018a964705d0bdcaf17bdf494e4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 17:25:34 +0200 Subject: [PATCH 342/409] Fix FreeBSD build --- cmake/freebsd/toolchain-x86_64.cmake | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmake/freebsd/toolchain-x86_64.cmake b/cmake/freebsd/toolchain-x86_64.cmake index 4635880b4a6..4d814693b39 100644 --- a/cmake/freebsd/toolchain-x86_64.cmake +++ b/cmake/freebsd/toolchain-x86_64.cmake @@ -8,4 +8,7 @@ set (CMAKE_CXX_COMPILER_TARGET "x86_64-pc-freebsd11") set (CMAKE_ASM_COMPILER_TARGET "x86_64-pc-freebsd11") set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/freebsd-x86_64") +# dprintf is used in a patched version of replxx +add_compile_definitions(_WITH_DPRINTF) + set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake From d16388000497251856f62e8ac67ade58c29f8e85 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 25 Aug 2024 00:11:31 -0400 Subject: [PATCH 343/409] process possible SSL error on connection reset --- base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 4873d259ae5..14c877b30af 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -311,6 +311,14 @@ int SecureSocketImpl::sendBytes(const void* buffer, int length, int flags) while (mustRetry(rc, remaining_time)); if (rc <= 0) { + // At this stage we still can have last not yet recieved SSL message containing SSL error + // so make a read to force SSL to process possible SSL error + if (SSL_get_error(_pSSL, rc) == SSL_ERROR_SYSCALL && SocketImpl::lastError() == POCO_ECONNRESET) + { + char c = 0; + SSL_read(_pSSL, &c, 1); + } + rc = handleError(rc); if (rc == 0) throw SSLConnectionUnexpectedlyClosedException(); } From f7cc3e9c59947af5b753b154f5b1b59d26fe67d4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 25 Aug 2024 00:13:12 -0400 Subject: [PATCH 344/409] postpone SSL handshake --- src/Client/Connection.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index e89bd7a2bf5..da6e5baa3ad 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -145,6 +145,9 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// work we need to pass host name separately. It will be send into TLS Hello packet to let /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). static_cast(socket.get())->setPeerHostName(host); + /// we want to postpone SSL handshake until first read or write operation + /// so any errors during negotiation would be properly processed + static_cast(socket.get())->setLazyHandshake(true); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "tcp_secure protocol is disabled because poco library was built without NetSSL support."); #endif From e4aceed36aa6a5dcc2b37d3a184af57c06cb202f Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 25 Aug 2024 07:48:42 +0000 Subject: [PATCH 345/409] Add virtual column _headers for url table engine --- base/poco/Net/include/Poco/Net/HTTPResponse.h | 4 ++ base/poco/Net/src/HTTPResponse.cpp | 9 +++++ src/IO/ReadWriteBufferFromHTTP.cpp | 14 +++++++ src/IO/ReadWriteBufferFromHTTP.h | 5 +++ src/Storages/StorageURL.cpp | 40 ++++++++++++++++--- src/Storages/StorageURL.h | 4 ++ ...3228_url_engine_response_headers.reference | 2 + .../03228_url_engine_response_headers.sql | 7 ++++ 8 files changed, 80 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03228_url_engine_response_headers.reference create mode 100644 tests/queries/0_stateless/03228_url_engine_response_headers.sql diff --git a/base/poco/Net/include/Poco/Net/HTTPResponse.h b/base/poco/Net/include/Poco/Net/HTTPResponse.h index 3c444c3d38c..623dfb783f3 100644 --- a/base/poco/Net/include/Poco/Net/HTTPResponse.h +++ b/base/poco/Net/include/Poco/Net/HTTPResponse.h @@ -18,7 +18,9 @@ #define Net_HTTPResponse_INCLUDED +#include #include + #include "Poco/Net/HTTPCookie.h" #include "Poco/Net/HTTPMessage.h" #include "Poco/Net/Net.h" @@ -180,6 +182,8 @@ namespace Net /// May throw an exception in case of a malformed /// Set-Cookie header. + void getHeaders(std::map & headers) const; + void write(std::ostream & ostr) const; /// Writes the HTTP response to the given /// output stream. diff --git a/base/poco/Net/src/HTTPResponse.cpp b/base/poco/Net/src/HTTPResponse.cpp index e0017321595..4d58d05a43e 100644 --- a/base/poco/Net/src/HTTPResponse.cpp +++ b/base/poco/Net/src/HTTPResponse.cpp @@ -209,6 +209,15 @@ void HTTPResponse::getCookies(std::vector& cookies) const } } +void HTTPResponse::getHeaders(std::map & headers) const +{ + headers.clear(); + for (const auto & it : *this) + { + headers.emplace(it.first, it.second); + } +} + void HTTPResponse::write(std::ostream& ostr) const { diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index a7bc0d4845c..826862bb5b3 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -443,6 +443,7 @@ std::unique_ptr ReadWriteBufferFromHTTP::initialize() } response.getCookies(cookies); + response.getHeaders(response_headers); content_encoding = response.get("Content-Encoding", ""); // Remember file size. It'll be used to report eof in next nextImpl() call. @@ -680,6 +681,19 @@ std::string ReadWriteBufferFromHTTP::getResponseCookie(const std::string & name, return def; } +Map ReadWriteBufferFromHTTP::getResponseHeaders() const +{ + Map map; + for (const auto & header : response_headers) + { + Tuple elem; + elem.emplace_back(header.first); + elem.emplace_back(header.second); + map.emplace_back(elem); + } + return map; +} + void ReadWriteBufferFromHTTP::setNextCallback(NextCallback next_callback_) { next_callback = next_callback_; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 1c9bda53008..cca22c64317 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -90,6 +90,9 @@ private: std::unique_ptr impl; std::vector cookies; + + std::map response_headers; + HTTPHeaderEntries http_header_entries; std::function next_callback; @@ -187,6 +190,8 @@ public: HTTPFileInfo getFileInfo(); static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); + + Map getResponseHeaders() const; }; using ReadWriteBufferFromHTTPPtr = std::unique_ptr; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6442891cf23..67b6ec4c9bf 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -44,10 +44,11 @@ #include #include +#include +#include +#include #include #include -#include -#include namespace ProfileEvents { @@ -167,7 +168,19 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings)); + auto virtual_columns_desc = VirtualColumnUtils::getVirtualsForFileLikeStorage( + storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings); + if (!storage_metadata.getColumns().has("_headers")) + { + virtual_columns_desc.addEphemeral( + "_headers", + std::make_shared( + std::make_shared(std::make_shared()), + std::make_shared(std::make_shared())), + ""); + } + + setVirtuals(virtual_columns_desc); } @@ -292,11 +305,13 @@ StorageURLSource::StorageURLSource( const URIParams & params, bool glob_url, bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false), WithContext(context_) + : SourceWithKeyCondition(info.source_header, false) + , WithContext(context_) , name(std::move(name_)) , columns_description(info.columns_description) , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) + , need_headers_virtual_column(info.requested_virtual_columns.contains("_headers")) + , requested_virtual_columns(info.requested_virtual_columns.eraseNames({"_headers"})) , block_for_format(info.format_header) , uri_iterator(uri_iterator_) , format(format_) @@ -436,6 +451,20 @@ Chunk StorageURLSource::generate() .path = curr_uri.getPath(), .size = current_file_size, }, getContext(), columns_description); + if (need_headers_virtual_column) + { + if (!http_response_headers_initialized) + { + http_response_headers = dynamic_cast(read_buf.get())->getResponseHeaders(); + http_response_headers_initialized = true; + } + + auto type = std::make_shared( + std::make_shared(std::make_shared()), + std::make_shared(std::make_shared())); + + chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)); + } return chunk; } @@ -446,6 +475,7 @@ Chunk StorageURLSource::generate() reader.reset(); input_format.reset(); read_buf.reset(); + http_response_headers_initialized = false; total_rows_in_file = 0; } return {}; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 19daf843431..6f1d544629a 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -220,6 +220,7 @@ private: String name; ColumnsDescription columns_description; NamesAndTypesList requested_columns; + bool need_headers_virtual_column; NamesAndTypesList requested_virtual_columns; Block block_for_format; std::shared_ptr uri_iterator; @@ -233,6 +234,9 @@ private: Poco::Net::HTTPBasicCredentials credentials; + Map http_response_headers; + bool http_response_headers_initialized = false; + std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference new file mode 100644 index 00000000000..f28952972b8 --- /dev/null +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -0,0 +1,2 @@ +Map(LowCardinality(String), LowCardinality(String)) +1 1 diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql new file mode 100644 index 00000000000..ff8e47611f4 --- /dev/null +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -0,0 +1,7 @@ +SELECT toTypeName(_headers) +FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); + +SELECT + *, + mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] +FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); From e1e692a968e5ccce64c43a90478903648f95bb27 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 25 Aug 2024 08:00:16 +0000 Subject: [PATCH 346/409] Fix and update docs --- docs/en/engines/table-engines/special/url.md | 1 + docs/en/sql-reference/table-functions/url.md | 1 + src/Storages/StorageURL.cpp | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index c906830d0e9..bbc28daef48 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -109,6 +109,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index b4027594e7c..7a354ea0364 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -54,6 +54,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`. ## Hive-style partitioning {#hive-style-partitioning} diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f7c3147d091..3fb00886540 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -465,7 +465,7 @@ Chunk StorageURLSource::generate() std::make_shared(std::make_shared()), std::make_shared(std::make_shared())); - chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)); + chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)->convertToFullColumnIfConst()); } return chunk; } From f38f95a144fa8840bc19647af3be9aa83a505196 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 25 Aug 2024 14:26:21 +0200 Subject: [PATCH 347/409] Update base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp --- base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 14c877b30af..eaf267d8a8b 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -311,7 +311,7 @@ int SecureSocketImpl::sendBytes(const void* buffer, int length, int flags) while (mustRetry(rc, remaining_time)); if (rc <= 0) { - // At this stage we still can have last not yet recieved SSL message containing SSL error + // At this stage we still can have last not yet received SSL message containing SSL error // so make a read to force SSL to process possible SSL error if (SSL_get_error(_pSSL, rc) == SSL_ERROR_SYSCALL && SocketImpl::lastError() == POCO_ECONNRESET) { From 9bcafbc90f66602749d3a66cb3631ebb50ce4483 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 10:46:18 +0800 Subject: [PATCH 348/409] improve_upper_lower --- src/Functions/LowerUpperUTF8Impl.h | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 490b53b0f4a..0b36caad817 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -4,20 +4,21 @@ #if USE_ICU -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; } template From 6c0102e971499bd44e4efe437f5e54028ba0ff53 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 16:28:56 +0800 Subject: [PATCH 349/409] fix failed uts --- src/Functions/LowerUpperUTF8Impl.h | 59 ++++++++----------- .../0_stateless/01431_utf8_ubsan.reference | 4 +- ...71_lower_upper_utf8_row_overlaps.reference | 4 +- 3 files changed, 28 insertions(+), 39 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 0b36caad817..5ba6c793d42 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -31,7 +31,7 @@ struct LowerUpperUTF8Impl ColumnString::Offsets & res_offsets, size_t input_rows_count) { - if (data.empty()) + if (input_rows_count == 0) return; bool all_ascii = isAllASCII(data.data(), data.size()); @@ -41,67 +41,56 @@ struct LowerUpperUTF8Impl return; } + res_data.resize(data.size()); + res_offsets.resize_exact(input_rows_count); UErrorCode error_code = U_ZERO_ERROR; - UCaseMap * csm = ucasemap_open(nullptr, 0, &error_code); + UCaseMap * case_map = ucasemap_open("", U_FOLD_CASE_DEFAULT, &error_code); if (U_FAILURE(error_code)) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Call ucasemap_open error:{}", u_errorName(error_code)); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Error calling ucasemap_open: {}", u_errorName(error_code)); - // String output; size_t curr_offset = 0; - res_data.resize(data.size()); - res_offsets.resize_exact(offsets.size()); - for (size_t i = 0; i < input_rows_count; ++i) + for (size_t row_i = 0; row_i < input_rows_count; ++row_i) { - const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); - size_t size = offsets[i] - offsets[i - 1] - 1; + const auto * src = reinterpret_cast(&data[offsets[row_i - 1]]); + size_t src_size = offsets[row_i] - offsets[row_i - 1] - 1; - int32_t out_size; + int32_t dst_size; if constexpr (upper) - out_size = ucasemap_utf8ToUpper( - csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + dst_size = ucasemap_utf8ToUpper( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); else - out_size = ucasemap_utf8ToLower( - csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + dst_size = ucasemap_utf8ToLower( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code // << std::endl; if (error_code == U_BUFFER_OVERFLOW_ERROR) { - size_t new_size = curr_offset + out_size + 1; + size_t new_size = curr_offset + dst_size + 1; res_data.resize(new_size); error_code = U_ZERO_ERROR; if constexpr (upper) - out_size = ucasemap_utf8ToUpper( - csm, - reinterpret_cast(&res_data[curr_offset]), - res_data.size() - curr_offset, - data_start, - size, - &error_code); + dst_size = ucasemap_utf8ToUpper( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); else - out_size = ucasemap_utf8ToLower( - csm, - reinterpret_cast(&res_data[curr_offset]), - res_data.size() - curr_offset, - data_start, - size, - &error_code); + dst_size = ucasemap_utf8ToLower( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); } if (error_code != U_ZERO_ERROR) throw DB::Exception( ErrorCodes::LOGICAL_ERROR, - "Call {} error:{} input:{} input_size:{}", + "Error calling {}: {} input: {} input_size: {}", upper ? "ucasemap_utf8ToUpper" : "ucasemap_utf8ToLower", u_errorName(error_code), - std::string_view(data_start, size), - size); + std::string_view(src, src_size), + src_size); - res_data[curr_offset + out_size] = 0; - curr_offset += out_size + 1; - res_offsets[i] = curr_offset; + res_data[curr_offset + dst_size] = 0; + curr_offset += dst_size + 1; + res_offsets[row_i] = curr_offset; } res_data.resize(curr_offset); diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.reference b/tests/queries/0_stateless/01431_utf8_ubsan.reference index dc785e57851..c98c950d535 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.reference +++ b/tests/queries/0_stateless/01431_utf8_ubsan.reference @@ -1,2 +1,2 @@ -EFBFBD -EFBFBD +FF +FF diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index deabef61a88..a3bac432482 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 From 53ace69ede72f106dc60f504e66a25ae402af6dc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 16:47:47 +0800 Subject: [PATCH 350/409] fix failed performance tests --- src/Functions/LowerUpperUTF8Impl.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5ba6c793d42..7d60bd54d2f 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -62,10 +62,8 @@ struct LowerUpperUTF8Impl else dst_size = ucasemap_utf8ToLower( case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); - // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code - // << std::endl; - if (error_code == U_BUFFER_OVERFLOW_ERROR) + if (error_code == U_BUFFER_OVERFLOW_ERROR || error_code == U_STRING_NOT_TERMINATED_WARNING) { size_t new_size = curr_offset + dst_size + 1; res_data.resize(new_size); 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 351/409] 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')" ) ) From 7d8942527355dedf19dc03246cb80ac5fc9e171a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 13:15:26 +0200 Subject: [PATCH 352/409] Speedup test 00653_verification_monotonic_data_load --- .../00653_verification_monotonic_data_load.sh | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index 7766d7720e1..2a26ec84582 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -47,38 +47,42 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2), CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --optimize_use_implicit_projections 0" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: UInt32 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> Int8" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> UInt8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait - -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Date -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') From aeac6c549873e27d180ad26ecc2572c268b528c9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 13:15:52 +0200 Subject: [PATCH 353/409] fix reference --- ...verification_monotonic_data_load.reference | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference b/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference index 8900af059b8..81260c0921c 100644 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference @@ -1,26 +1,26 @@ -no monotonic int case: String -> UInt64 -no monotonic int case: FixedString -> UInt64 -monotonic int case: Int32 -> Int64 -monotonic int case: Int32 -> UInt64 -monotonic int case: Int32 -> Int32 -monotonic int case: Int32 -> UInt32 -monotonic int case: Int32 -> Int16 -monotonic int case: Int32 -> UInt16 -monotonic int case: UInt32 -> Int64 -monotonic int case: UInt32 -> UInt64 -monotonic int case: UInt32 -> Int32 -monotonic int case: UInt32 -> UInt32 -monotonic int case: UInt32 -> Int16 -monotonic int case: UInt32 -> UInt16 -monotonic int case: Enum16 -> Int32 -monotonic int case: Enum16 -> UInt32 -monotonic int case: Enum16 -> Int16 -monotonic int case: Enum16 -> UInt16 -monotonic int case: Enum16 -> Int8 -monotonic int case: Enum16 -> UInt8 -monotonic int case: Date -> Int32 -monotonic int case: Date -> UInt32 -monotonic int case: Date -> Int16 -monotonic int case: Date -> UInt16 -monotonic int case: Date -> Int8 -monotonic int case: Date -> UInt8 +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK From 91d14e6631adbc071dbd20e2129c2b3db2797f90 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:49:32 +0200 Subject: [PATCH 354/409] Update 03230_json_alias_new_old_types.sql --- tests/queries/0_stateless/03230_json_alias_new_old_types.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql index 97d64f2a606..06d4790e0f9 100644 --- a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql @@ -1,7 +1,7 @@ set allow_experimental_object_type=1; set allow_experimental_json_type=1; set use_json_alias_for_old_object_type=0; -select '{"a" : 42}'::JSON as json, toTypeName(json); +select materialize('{"a" : 42}')::JSON as json, toTypeName(json); set use_json_alias_for_old_object_type=1; select '{"a" : 42}'::JSON as json, toTypeName(json); select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS} From f1130f3130d3e4702ccd7f8fce493e49ff79ea4b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:49:45 +0200 Subject: [PATCH 355/409] Update 03230_json_alias_new_old_types.reference --- .../0_stateless/03230_json_alias_new_old_types.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference index 91fd15697be..f03e0117618 100644 --- a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference @@ -1,2 +1,2 @@ -{"a":42} JSON +{"a":"42"} JSON {"a":42} Object(\'json\') From 77061db95595cea33c2e5f84804c1f9a799ec6d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 14:37:02 +0000 Subject: [PATCH 356/409] Adding comments and checks. --- .../QueryPlan/MergingAggregatedStep.cpp | 13 +++++++++++++ .../Transforms/MergingAggregatedTransform.cpp | 16 +++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 8332ad73df6..d35c38a4e32 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static bool memoryBoundMergingWillBeUsed( const DataStream & input_stream, bool memory_bound_merging_of_aggregation_results_enabled, @@ -93,6 +98,10 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (memoryBoundMergingWillBeUsed()) { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory bound merging of aggregated results is not supported for grouping sets."); + auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), @@ -123,6 +132,10 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (!memory_efficient_aggregation) { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory efficient merging of aggregated results is not supported for grouping sets."); + /// We union several sources into one, paralleling the work. pipeline.resize(1); diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 114a32b3d83..99fbf3bf4f0 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -13,6 +13,10 @@ namespace ErrorCodes Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) { + /// __grouping_set is neigher GROUP BY key nor an aggregate function. + /// It behaves like a GROUP BY key, but we cannot append it to keys + /// because it changes hashing method and buckets for two level aggregation. + /// Now, this column is processed "manually" by merging each group separately. if (in_header.has("__grouping_set")) out_header.insert(0, in_header.getByName("__grouping_set")); @@ -39,32 +43,41 @@ void MergingAggregatedTransform::addBlock(Block block) auto grouping_column = block.getByPosition(grouping_position).column; block.erase(grouping_position); + /// Split a block by __grouping_set values. + const auto * grouping_column_typed = typeid_cast(grouping_column.get()); if (!grouping_column_typed) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); - const auto & grouping_data = grouping_column_typed->getData(); + /// Enumerate groups and fill the selector. std::map enumerated_groups; IColumn::Selector selector; + const auto & grouping_data = grouping_column_typed->getData(); size_t num_rows = grouping_data.size(); UInt64 last_group = grouping_data[0]; for (size_t row = 1; row < num_rows; ++row) { auto group = grouping_data[row]; + + /// Optimization for equal ranges. if (last_group == group) continue; + /// Optimization for single group. if (enumerated_groups.empty()) { selector.reserve(num_rows); enumerated_groups.emplace(last_group, enumerated_groups.size()); } + /// Fill the last equal range. selector.resize_fill(row, enumerated_groups[last_group]); + /// Enumerate new group if did not see it before. enumerated_groups.emplace(last_group, enumerated_groups.size()); } + /// Optimization for single group. if (enumerated_groups.empty()) { auto & bucket_to_blocks = grouping_sets[last_group]; @@ -72,6 +85,7 @@ void MergingAggregatedTransform::addBlock(Block block) return; } + /// Fill the last equal range. selector.resize_fill(num_rows, enumerated_groups[last_group]); const size_t num_groups = enumerated_groups.size(); From 42e7cc476e4e733839370681366ffde64185ba6c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 14:48:29 +0000 Subject: [PATCH 357/409] Fixing typos. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 99fbf3bf4f0..9b107b70075 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) { - /// __grouping_set is neigher GROUP BY key nor an aggregate function. + /// __grouping_set is neither GROUP BY key nor an aggregate function. /// It behaves like a GROUP BY key, but we cannot append it to keys /// because it changes hashing method and buckets for two level aggregation. /// Now, this column is processed "manually" by merging each group separately. From e7d17573e1a005c015ba0178cb02696084774ba7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 16:55:45 +0200 Subject: [PATCH 358/409] Add assert src/Storages/StorageURL.cpp --- src/Storages/StorageURL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 3fb00886540..1c7439f9a55 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -453,6 +453,7 @@ Chunk StorageURLSource::generate() .size = current_file_size, }, getContext()); + chassert(dynamic_cast(read_buf.get())); if (need_headers_virtual_column) { if (!http_response_headers_initialized) From 5f587af078eb6f9c962ee1ba0dccfefcab400f3a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 15:15:16 +0000 Subject: [PATCH 359/409] Review fix. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 9b107b70075..78fb2f340bf 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -74,7 +74,9 @@ void MergingAggregatedTransform::addBlock(Block block) /// Fill the last equal range. selector.resize_fill(row, enumerated_groups[last_group]); /// Enumerate new group if did not see it before. - enumerated_groups.emplace(last_group, enumerated_groups.size()); + enumerated_groups.emplace(group, enumerated_groups.size()); + + last_group = group; } /// Optimization for single group. From 9d9ef691968f4d93bc90bad9624af2b3390b98e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 15:21:48 +0000 Subject: [PATCH 360/409] Fixing check. --- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index d35c38a4e32..7207b5e6c7f 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -132,10 +132,6 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (!memory_efficient_aggregation) { - if (input_streams.front().header.has("__grouping_set")) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Memory efficient merging of aggregated results is not supported for grouping sets."); - /// We union several sources into one, paralleling the work. pipeline.resize(1); @@ -145,6 +141,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c } else { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory efficient merging of aggregated results is not supported for grouping sets."); auto num_merge_threads = memory_efficient_merge_threads ? memory_efficient_merge_threads : max_threads; From c27513e540ae929264424daa59790894c0cfde3c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 26 Aug 2024 17:38:07 +0200 Subject: [PATCH 361/409] Revert "Fix prewhere without columns and without adaptive index granularity (almost w/o anything)" --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 11 ---- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +---- .../02967_prewhere_no_columns.reference | 2 - .../0_stateless/02967_prewhere_no_columns.sql | 51 ------------------- ...n_adaptive_granularity_no_errors.reference | 2 - ...198_non_adaptive_granularity_no_errors.sql | 12 ----- .../00166_explain_estimate.reference | 4 +- 7 files changed, 3 insertions(+), 91 deletions(-) delete mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.reference delete mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.sql delete mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference delete mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..a1565173b47 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1334,17 +1334,6 @@ void IMergeTreeDataPart::loadRowsCount() auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); - - if (!index_granularity.empty() && rows_count < index_granularity.getTotalRows() && index_granularity_info.fixed_index_granularity) - { - /// Adjust last granule size to match the number of rows in the part in case of fixed index_granularity. - index_granularity.popMark(); - index_granularity.appendMark(rows_count % index_granularity_info.fixed_index_granularity); - if (rows_count != index_granularity.getTotalRows()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Index granularity total rows in part {} does not match rows_count: {}, instead of {}", - name, index_granularity.getTotalRows(), rows_count); - } }; if (index_granularity.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 3edcce74b09..8b6735e0fe2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -577,10 +577,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) { - /// With fixed granularity we can have last mark with less rows than granularity - const bool is_last_mark = (mark_num + 1 == index_granularity.getMarksCount()); - if (!index_granularity_info.fixed_index_granularity || !is_last_mark) - throw Exception( + throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", @@ -844,14 +841,7 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_ /// Without offset rows_written_in_last_mark = 0; } - - if (compute_granularity) - { - index_granularity.popMark(); - index_granularity.appendMark(new_rows_in_last_mark); - } } - } } diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.reference b/tests/queries/0_stateless/02967_prewhere_no_columns.reference deleted file mode 100644 index df105254618..00000000000 --- a/tests/queries/0_stateless/02967_prewhere_no_columns.reference +++ /dev/null @@ -1,2 +0,0 @@ -105 -105 diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.sql b/tests/queries/0_stateless/02967_prewhere_no_columns.sql deleted file mode 100644 index efcc952caa2..00000000000 --- a/tests/queries/0_stateless/02967_prewhere_no_columns.sql +++ /dev/null @@ -1,51 +0,0 @@ -CREATE TABLE t_02967 -( - `key` Date, - `value` UInt16 -) -ENGINE = MergeTree -ORDER BY key -SETTINGS - index_granularity_bytes = 0 --8192 --, min_index_granularity_bytes = 2 - , index_granularity = 100 - , min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0 --- --- , min_bytes_for_wide_part = 2 -AS SELECT - number, - repeat(toString(number), 5) -FROM numbers(105.); - - - --- Check with newly inserted data part. It's in-memory structured are filled at insert time. -SELECT - count(ignore(*)) -FROM t_02967 -PREWHERE CAST(ignore() + 1 as UInt8) -GROUP BY - ignore(65535, *), - ignore(255, 256, *) -SETTINGS - --send_logs_level='test', - max_threads=1; - - - --- Reload part form disk to check that in-meory structures where properly serilaized-deserialized -DETACH TABLE t_02967; -ATTACH TABLE t_02967; - - -SELECT - count(ignore(*)) -FROM t_02967 -PREWHERE CAST(ignore() + 1 as UInt8) -GROUP BY - ignore(65535, *), - ignore(255, 256, *) -SETTINGS - --send_logs_level='test', - max_threads=1; - -DROP TABLE t_02967; diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference deleted file mode 100644 index fcd78da1283..00000000000 --- a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference +++ /dev/null @@ -1,2 +0,0 @@ -1000000 -1000000 diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql deleted file mode 100644 index 25798ef6d33..00000000000 --- a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS data_02051__fuzz_24; - -CREATE TABLE data_02051__fuzz_24 (`key` Int16, `value` String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part=0 AS SELECT number, repeat(toString(number), 5) FROM numbers(1000000.); - -SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); - -detach table data_02051__fuzz_24; -attach table data_02051__fuzz_24; - -SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); - -DROP TABLE data_02051__fuzz_24; diff --git a/tests/queries/1_stateful/00166_explain_estimate.reference b/tests/queries/1_stateful/00166_explain_estimate.reference index 85ecd0b9a71..71ddd681581 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.reference +++ b/tests/queries/1_stateful/00166_explain_estimate.reference @@ -1,5 +1,5 @@ test hits 1 57344 7 -test hits 1 8832938 1079 -test hits 1 829354 102 +test hits 1 8839168 1079 +test hits 1 835584 102 test hits 1 8003584 977 test hits 2 581632 71 From cf58ce217a34296c7e3332e0fe147f51fb885959 Mon Sep 17 00:00:00 2001 From: Jordi Date: Mon, 26 Aug 2024 18:49:01 +0200 Subject: [PATCH 362/409] Add system.projections table --- .../operations/system-tables/projections.md | 41 +++ .../System/StorageSystemProjections.cpp | 274 ++++++++++++++++++ .../System/StorageSystemProjections.h | 30 ++ src/Storages/System/attachSystemTables.cpp | 2 + .../03230_system_projections.reference | 6 + .../0_stateless/03230_system_projections.sql | 39 +++ 6 files changed, 392 insertions(+) create mode 100644 docs/en/operations/system-tables/projections.md create mode 100644 src/Storages/System/StorageSystemProjections.cpp create mode 100644 src/Storages/System/StorageSystemProjections.h create mode 100644 tests/queries/0_stateless/03230_system_projections.reference create mode 100644 tests/queries/0_stateless/03230_system_projections.sql diff --git a/docs/en/operations/system-tables/projections.md b/docs/en/operations/system-tables/projections.md new file mode 100644 index 00000000000..e9c88f731b2 --- /dev/null +++ b/docs/en/operations/system-tables/projections.md @@ -0,0 +1,41 @@ +--- +slug: /en/operations/system-tables/projections +--- +# projections + +Contains information about existing projections in all the tables. + +Columns: + +- `database` ([String](../../sql-reference/data-types/string.md)) — Database name. +- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `name` ([String](../../sql-reference/data-types/string.md)) — Projection name. +- `type` ([Enum](../../sql-reference/data-types/enum.md)) — Projection type ('Normal' = 0, 'Aggregate' = 1). +- `sorting_key` ([Array(String)](../../sql-reference/data-types/array.md)) — Projection sorting key. +- `query` ([String](../../sql-reference/data-types/string.md)) — Projection query. + +**Example** + +```sql +SELECT * FROM system.projections LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: default +table: landing +name: improved_sorting_key +type: Normal +sorting_key: ['user_id','date'] +query: SELECT * ORDER BY user_id, date + +Row 2: +────── +database: default +table: landing +name: agg_no_key +type: Aggregate +sorting_key: [] +query: SELECT count() +``` diff --git a/src/Storages/System/StorageSystemProjections.cpp b/src/Storages/System/StorageSystemProjections.cpp new file mode 100644 index 00000000000..06c00d764d7 --- /dev/null +++ b/src/Storages/System/StorageSystemProjections.cpp @@ -0,0 +1,274 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +StorageSystemProjections::StorageSystemProjections(const StorageID & table_id_) + : IStorage(table_id_) +{ + auto projection_type_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Normal", static_cast(ProjectionDescription::Type::Normal)}, + {"Aggregate", static_cast(ProjectionDescription::Type::Aggregate)} + } + ); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( + { + { "database", std::make_shared(), "Database name."}, + { "table", std::make_shared(), "Table name."}, + { "name", std::make_shared(), "Projection name."}, + { "type", std::move(projection_type_datatype), "Projection type."}, + { "sorting_key", std::make_shared(std::make_shared()), "Projection sorting key."}, + { "query", std::make_shared(), "Projection query."}, + })); + setInMemoryMetadata(storage_metadata); +} + +class ProjectionsSource : public ISource +{ +public: + ProjectionsSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ContextPtr context_) + : ISource(header) + , column_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + , database_idx(0) + {} + + String getName() const override { return "Projections"; } + +protected: + Chunk generate() override + { + if (database_idx >= databases->size()) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (database) + break; + ++database_idx; + } + + if (database_idx >= databases->size()) + break; + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + const auto table = tables_it->table(); + if (!table) + continue; + StorageMetadataPtr metadata_snapshot = table->getInMemoryMetadataPtr(); + if (!metadata_snapshot) + continue; + const auto & projections = metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + // 'database' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + // 'table' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + // 'name' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(projection.name); + // 'type' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(projection.type); + // 'sorting_key' column + if (column_mask[src_index++]) { + auto columns = projection.metadata->getSortingKeyColumns(); + + Array sorting_key; + sorting_key.reserve(columns.size()); + for (const auto & column : columns) + { + sorting_key.push_back(column); + } + res_columns[res_index++]->insert(sorting_key); + } + // 'query' column + if (column_mask[src_index++]) { + res_columns[res_index++]->insert(serializeAST(*projection.definition_ast->children.at(0))); + } + } + } + } + return Chunk(std::move(res_columns), rows_count); + } + +private: + std::vector column_mask; + UInt64 max_block_size; + ColumnPtr databases; + ContextPtr context; + size_t database_idx; + DatabasePtr database; + std::string database_name; + DatabaseTablesIteratorPtr tables_it; +}; + +class ReadFromSystemProjections : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemProjections"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemProjections( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + std::vector columns_mask; + const size_t max_block_size; + ExpressionActionsPtr virtual_columns_filter; +}; + +void ReadFromSystemProjections::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + if (filter_actions_dag) + { + Block block_to_filter + { + { ColumnString::create(), std::make_shared(), "database" }, + }; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + if (dag) + virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context); + } +} + +void StorageSystemProjections::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /* processed_stage */, + size_t max_block_size, + size_t /* num_streams */) +{ + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemProjections::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & [database_name, database] : databases) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; + + /// Lazy database can contain only very primitive tables, + /// it cannot contain tables with data skipping indices. + /// Skip it to avoid unnecessary tables loading in the Lazy database. + if (database->getEngineName() != "Lazy") + column->insert(database_name); + } + + /// Condition on "database" in a query acts like an index. + Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, block); + + ColumnPtr & filtered_databases = block.getByPosition(0).column; + pipeline.init(Pipe(std::make_shared( + std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases), context))); +} + +} diff --git a/src/Storages/System/StorageSystemProjections.h b/src/Storages/System/StorageSystemProjections.h new file mode 100644 index 00000000000..46e73cec6cd --- /dev/null +++ b/src/Storages/System/StorageSystemProjections.h @@ -0,0 +1,30 @@ +#pragma once + +#include + + +namespace DB +{ + +/// For system.projections table - describes the projections in tables, similar to system.data_skipping_indices. +class StorageSystemProjections : public IStorage +{ +public: + explicit StorageSystemProjections(const StorageID & table_id_); + + std::string getName() const override { return "StorageSystemProjections"; } + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + bool isSystemStorage() const override { return true; } +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 97eda1db3fa..816ba5095b1 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include #include @@ -166,6 +167,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "errors", "Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace."); attach(context, system_database, "warnings", "Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server."); attachNoDescription(context, system_database, "data_skipping_indices", "Contains all the information about all the data skipping indices in tables, similar to system.columns."); + attachNoDescription(context, system_database, "projections", "Contains all the information about all the projections in tables, similar to system.data_skipping_indices."); attach(context, system_database, "licenses", "Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources."); attach(context, system_database, "time_zones", "Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse."); attach(context, system_database, "backups", "Contains a list of all BACKUP or RESTORE operations with their current states and other propertis. Note, that table is not persistent and it shows only operations executed after the last server restart."); diff --git a/tests/queries/0_stateless/03230_system_projections.reference b/tests/queries/0_stateless/03230_system_projections.reference new file mode 100644 index 00000000000..22ca272955c --- /dev/null +++ b/tests/queries/0_stateless/03230_system_projections.reference @@ -0,0 +1,6 @@ +default projections improved_sorting_key Normal ['d1','key'] SELECT * ORDER BY d1, key +default projections_2 agg Aggregate ['name'] SELECT name, max(frequency) AS max_frequency GROUP BY name +default projections_2 agg_no_key Aggregate [] SELECT max(frequency) AS max_frequency +1 +2 +improved_sorting_key diff --git a/tests/queries/0_stateless/03230_system_projections.sql b/tests/queries/0_stateless/03230_system_projections.sql new file mode 100644 index 00000000000..37c1e5df8ef --- /dev/null +++ b/tests/queries/0_stateless/03230_system_projections.sql @@ -0,0 +1,39 @@ +DROP TABLE IF EXISTS projections; +DROP TABLE IF EXISTS projections_2; + +CREATE TABLE projections +( + key String, + d1 Int, + PROJECTION improved_sorting_key ( + SELECT * + ORDER BY d1, key + ) +) +Engine=MergeTree() +ORDER BY key; + +CREATE TABLE projections_2 +( + name String, + frequency UInt64, + PROJECTION agg ( + SELECT name, max(frequency) max_frequency + GROUP BY name + ), + PROJECTION agg_no_key ( + SELECT max(frequency) max_frequency + ) +) +Engine=MergeTree() +ORDER BY name; + +SELECT * FROM system.projections WHERE database = currentDatabase(); + +SELECT count(*) FROM system.projections WHERE table = 'projections' AND database = currentDatabase(); +SELECT count(*) FROM system.projections WHERE table = 'projections_2' AND database = currentDatabase(); + +SELECT name FROM system.projections WHERE type = 'Normal' AND database = currentDatabase(); + +DROP TABLE projections; +DROP TABLE projections_2; \ No newline at end of file From 13593340f241ca80a213bb2a16c81912e04bbb1d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:24:00 +0200 Subject: [PATCH 363/409] Add checks --- tests/integration/runner | 45 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/tests/integration/runner b/tests/integration/runner index 0667541b196..0f1e466051f 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -6,6 +6,7 @@ import logging import os import random import shlex +import shutil import signal import string import subprocess @@ -135,6 +136,47 @@ def check_args_and_update_paths(args): ) +def check_iptables_legacy(): + iptables_path = shutil.which("iptables") + + if iptables_path is None: + print("Error: 'iptables' not found in PATH") + sys.exit(1) # Exit with error code 1 + + try: + file_info = os.stat(iptables_path) + file_info_str = str(file_info) + + if "legacy" in file_info_str: + print( + "iptables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + ) + sys.exit(1) + + except FileNotFoundError: + print(f"Error: '{iptables_path}' not found") + sys.exit(1) + + +def check_iptables_forward_accept(ipv6=False): + command = "iptables" + if ipv6: + command = "ip6tables" + output = os.popen(command + "-S FORWARD").read() + + # Check if the output contains '-P FORWARD ACCEPT' + if "-P FORWARD ACCEPT" not in output: + print( + f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" + ) + + +def chech_prerequisites(): + check_iptables_legacy() + check_iptables_forward_accept(ipv6=False) + check_iptables_forward_accept(ipv6=True) + + def docker_kill_handler_handler(signum, frame): _, _ = signum, frame subprocess.check_call( @@ -163,6 +205,9 @@ if __name__ == "__main__": level=logging.INFO, format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)", ) + + chech_prerequisites() + parser = argparse.ArgumentParser(description="ClickHouse integration tests runner") parser.add_argument( From df7d44bc3adf2fecbd8506e155d0016a9847f7f3 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:27:07 +0200 Subject: [PATCH 364/409] Lint --- tests/integration/runner | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 0f1e466051f..a0e85ea9de6 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,7 +141,7 @@ def check_iptables_legacy(): if iptables_path is None: print("Error: 'iptables' not found in PATH") - sys.exit(1) # Exit with error code 1 + sys.exit(1) try: file_info = os.stat(iptables_path) @@ -164,7 +164,6 @@ def check_iptables_forward_accept(ipv6=False): command = "ip6tables" output = os.popen(command + "-S FORWARD").read() - # Check if the output contains '-P FORWARD ACCEPT' if "-P FORWARD ACCEPT" not in output: print( f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" From 07cb5b27eb0f4ef57e507b6ab04ca97585900143 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:41:56 +0200 Subject: [PATCH 365/409] Lint --- tests/integration/runner | 47 +++++++++++++++++++++++----------------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index a0e85ea9de6..2224291f3fb 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -138,10 +138,13 @@ def check_args_and_update_paths(args): def check_iptables_legacy(): iptables_path = shutil.which("iptables") + ip6tables_path = shutil.which("ip6tables") if iptables_path is None: print("Error: 'iptables' not found in PATH") sys.exit(1) + if ip6tables_path is None: + print("Error: 'ip6tables' not found in PATH, ignoring") try: file_info = os.stat(iptables_path) @@ -153,29 +156,23 @@ def check_iptables_legacy(): ) sys.exit(1) + if not ip6tables_path: + return + + file_info = os.stat(ip6tables_path) + file_info_str = str(file_info) + + if "legacy" in file_info_str: + print( + "ip6tables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + ) + sys.exit(1) + except FileNotFoundError: print(f"Error: '{iptables_path}' not found") sys.exit(1) -def check_iptables_forward_accept(ipv6=False): - command = "iptables" - if ipv6: - command = "ip6tables" - output = os.popen(command + "-S FORWARD").read() - - if "-P FORWARD ACCEPT" not in output: - print( - f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" - ) - - -def chech_prerequisites(): - check_iptables_legacy() - check_iptables_forward_accept(ipv6=False) - check_iptables_forward_accept(ipv6=True) - - def docker_kill_handler_handler(signum, frame): _, _ = signum, frame subprocess.check_call( @@ -205,8 +202,6 @@ if __name__ == "__main__": format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)", ) - chech_prerequisites() - parser = argparse.ArgumentParser(description="ClickHouse integration tests runner") parser.add_argument( @@ -355,12 +350,24 @@ if __name__ == "__main__": help="Bind volume to this dir to use for dockerd files", ) + parser.add_argument( + "--ignore-iptables-legacy-check", + action="store_true", + default=False, + help="Ignore iptables-legacy usage check", + ) + parser.add_argument("pytest_args", nargs="*", help="args for pytest command") args = parser.parse_args() check_args_and_update_paths(args) + if not args.ignore_iptables_legacy_check: + check_iptables_legacy() + else: + logging.warning("Skipping iptables-legacy check") + parallel_args = "" if args.parallel: parallel_args += "--dist=loadfile" From 936bbe7d0d1da9c5ddecaf04eff45c3805d0429b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Aug 2024 20:02:03 +0200 Subject: [PATCH 366/409] handle trash parts correctly --- src/Storages/MergeTree/MergeList.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index ed58b29d584..b80d7fccc91 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -33,12 +34,12 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; /// FIXME why do we need a merge list element for projection parts at all? - bool skip_sanity_checks = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION; + bool is_fake_projection_part = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION; size_t normal_parts_count = 0; for (const auto & source_part : future_part->parts) { - if (!skip_sanity_checks && !source_part->getParentPart()) + if (!is_fake_projection_part && !source_part->getParentPart()) { ++normal_parts_count; if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) @@ -57,14 +58,14 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (!future_part->parts.empty()) { source_data_version = future_part->parts[0]->info.getDataVersion(); - is_mutation = (result_part_info.level == future_part->parts[0]->info.level); + is_mutation = (result_part_info.level == future_part->parts[0]->info.level) && !is_fake_projection_part; WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; part->partition.serializeText(part->storage, out, {}); } - if (!skip_sanity_checks && is_mutation && normal_parts_count != 1) + if (!is_fake_projection_part && is_mutation && normal_parts_count != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}: {}", future_part->parts.size(), result_part_info.getPartNameV1(), fmt::join(source_part_names, ", ")); From c43fcce77da699012719a3489f6d71b3be3ce88d Mon Sep 17 00:00:00 2001 From: Jordi Date: Mon, 26 Aug 2024 20:04:19 +0200 Subject: [PATCH 367/409] Fix style check --- src/Storages/System/StorageSystemProjections.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/System/StorageSystemProjections.cpp b/src/Storages/System/StorageSystemProjections.cpp index 06c00d764d7..7082853e4f9 100644 --- a/src/Storages/System/StorageSystemProjections.cpp +++ b/src/Storages/System/StorageSystemProjections.cpp @@ -135,7 +135,8 @@ protected: if (column_mask[src_index++]) res_columns[res_index++]->insert(projection.type); // 'sorting_key' column - if (column_mask[src_index++]) { + if (column_mask[src_index++]) + { auto columns = projection.metadata->getSortingKeyColumns(); Array sorting_key; @@ -147,7 +148,8 @@ protected: res_columns[res_index++]->insert(sorting_key); } // 'query' column - if (column_mask[src_index++]) { + if (column_mask[src_index++]) + { res_columns[res_index++]->insert(serializeAST(*projection.definition_ast->children.at(0))); } } @@ -254,8 +256,7 @@ void ReadFromSystemProjections::initializePipeline(QueryPipelineBuilder & pipeli if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; - /// Lazy database can contain only very primitive tables, - /// it cannot contain tables with data skipping indices. + /// Lazy database can contain only very primitive tables, it cannot contain tables with projections. /// Skip it to avoid unnecessary tables loading in the Lazy database. if (database->getEngineName() != "Lazy") column->insert(database_name); From fb3c4671f02edfc2e4e79610e643164c288dcffb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 20:45:25 +0200 Subject: [PATCH 368/409] Improve error messages --- tests/integration/runner | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 2224291f3fb..cdd75438b87 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -152,7 +152,12 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( - "iptables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + """ + iptables is in 'legacy' mode. This is not supported. + + Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. + Or, use --ignore-iptables-legacy-check. + """ ) sys.exit(1) @@ -164,7 +169,12 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( - "ip6tables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + """ + ip6tables is in 'legacy' mode. This is not supported. + + Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. + Or, use --ignore-iptables-legacy-check. + """ ) sys.exit(1) From 89359c1e6b22d8a0a2ad5e56c5201c8b99c72b67 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 00:53:33 +0000 Subject: [PATCH 369/409] Make long_parquet* tests less long --- .../queries/0_stateless/00900_long_parquet.sh | 174 +++++++++--------- .../0_stateless/00900_long_parquet_decimal.sh | 135 +++++++------- .../0_stateless/00900_long_parquet_load.sh | 12 +- 3 files changed, 167 insertions(+), 154 deletions(-) diff --git a/tests/queries/0_stateless/00900_long_parquet.sh b/tests/queries/0_stateless/00900_long_parquet.sh index 5d6317b2787..07d2f24e446 100755 --- a/tests/queries/0_stateless/00900_long_parquet.sh +++ b/tests/queries/0_stateless/00900_long_parquet.sh @@ -8,58 +8,64 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS contributors; + CREATE TABLE contributors (name String) ENGINE = Memory;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT Parquet" -# random results -${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null -${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors" +${CLICKHOUSE_CLIENT} -n --query=" + -- random results + SELECT * FROM contributors LIMIT 10 FORMAT Null; + DROP TABLE contributors; -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_numbers" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_numbers; + CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory;" # less than default block size (65k) ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + TRUNCATE TABLE parquet_numbers;" # More than default block size ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + TRUNCATE TABLE parquet_numbers;" ${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" + TRUNCATE TABLE parquet_numbers;" ${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + DROP TABLE parquet_numbers; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_numbers" - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_events" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_events; + CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_events FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_events" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event; + DROP TABLE parquet_events; -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types4" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" -# convert min type -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory" -# convert max type -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_types1; + DROP TABLE IF EXISTS parquet_types2; + DROP TABLE IF EXISTS parquet_types3; + DROP TABLE IF EXISTS parquet_types4; + CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; + CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; + -- convert min type + CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory; + -- convert max type + CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9))" + INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9)); -# min -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9))" + -- min + INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9)); -# max -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9))" + -- max + INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9));" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" @@ -72,8 +78,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8 FORMAT echo diff: diff "${CLICKHOUSE_TMP}"/parquet_all_types_1.dump "${CLICKHOUSE_TMP}"/parquet_all_types_2.dump -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9))" +${CLICKHOUSE_CLIENT} -n --query=" + TRUNCATE TABLE parquet_types2; + INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9));" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 FORMAT Parquet" @@ -81,70 +88,69 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80, ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 FORMAT Parquet" -echo dest: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8" -echo min: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8" -echo max: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT 'dest:'; + SELECT * FROM parquet_types2 ORDER BY int8; + SELECT 'min:'; + SELECT * FROM parquet_types3 ORDER BY int8; + SELECT 'max:'; + SELECT * FROM parquet_types4 ORDER BY int8; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types5" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types6" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)" + DROP TABLE IF EXISTS parquet_types5; + DROP TABLE IF EXISTS parquet_types6; + TRUNCATE TABLE parquet_types2; + CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory; + CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory; + INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet" echo dest from null: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types6 ORDER BY int8" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_types6 ORDER BY int8; + DROP TABLE parquet_types5; + DROP TABLE parquet_types6; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types5" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types6" + DROP TABLE parquet_types1; + DROP TABLE parquet_types2; + DROP TABLE parquet_types3; + DROP TABLE parquet_types4; + DROP TABLE IF EXISTS parquet_arrays; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types3" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4" + CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory(); + INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001]); -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays" - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [])" + INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], []);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays ORDER BY id" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_arrays ORDER BY id; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_arrays" + DROP TABLE parquet_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory()" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], [])" + DROP TABLE IF EXISTS parquet_nullable_arrays; + CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory(); + INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], []);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays ORDER BY id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nullable_arrays" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_nullable_arrays ORDER BY id; + DROP TABLE parquet_nullable_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nested_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory() " -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []])" + DROP TABLE IF EXISTS parquet_nested_arrays; + CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory(); + INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []]);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nested_arrays" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_nested_arrays; + DROP TABLE parquet_nested_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_decimal" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123)" + DROP TABLE IF EXISTS parquet_decimal; + CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory; + INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_decimal FORMAT Arrow" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_decimal" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_decimal; + DROP TABLE parquet_decimal;" diff --git a/tests/queries/0_stateless/00900_long_parquet_decimal.sh b/tests/queries/0_stateless/00900_long_parquet_decimal.sh index 1b82ae9bbc7..a819dcbcdc3 100755 --- a/tests/queries/0_stateless/00900_long_parquet_decimal.sh +++ b/tests/queries/0_stateless/00900_long_parquet_decimal.sh @@ -6,108 +6,115 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" -# Simple small values -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; + + -- Simple small values + CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100); + INSERT INTO decimal (a, b, c) VALUES (1, 1, 1); + INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal0.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump echo diff0: diff "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (0.000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-0.000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');" + CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999); + INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999); + INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999); + INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999); + INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999); + INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999); + INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999); + INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999); + INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999); + INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999); + INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001); + INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001); + INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001); + INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001); + INSERT INTO decimal (i) VALUES (0.000000000000000001); + INSERT INTO decimal (i) VALUES (-0.000000000000000001); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0); + INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990'); + INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38'); + INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal1.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump echo diff1: diff "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);" + CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal2.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump echo diff2: diff "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" + CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" # Empty table test # throws No data to insert ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" 2> /dev/null echo nothing: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM decimal2 ORDER BY a, b, c, d; + TRUNCATE TABLE decimal2; -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, Null, Null)" + INSERT INTO decimal VALUES (Null, Null, Null, Null);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" echo nulls: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM decimal2 ORDER BY a, b, c, d; + TRUNCATE TABLE decimal2; -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (1, Null, Null, Null)" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, 1, Null, Null)" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, 1, Null)" + INSERT INTO decimal VALUES (1, Null, Null, Null); + INSERT INTO decimal VALUES (Null, 1, Null, Null); + INSERT INTO decimal VALUES (Null, Null, 1, Null);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_3.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" -echo full orig: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d;" -echo full inserted: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT 'full orig:'; + SELECT * FROM decimal ORDER BY a, b, c, d; + SELECT 'full inserted:'; + SELECT * FROM decimal2 ORDER BY a, b, c, d;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump echo diff3: diff "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2;" diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 3a7022ac0cf..0a7f10fe16d 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -57,14 +57,14 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A # COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.py $JSON` 2>&1 || continue COLUMNS=$(cat "$COLUMNS_FILE") || continue - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" - $CLICKHOUSE_CLIENT <&1 | sed 's/Exception/Ex---tion/' - ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100" - ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load" + ${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100; + DROP TABLE parquet_load;" done From 1aebcc1cb7b90416c7a85a40d915441efbe1c98a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 11:43:15 +0200 Subject: [PATCH 370/409] Fix 01114_database_atomic flakiness --- .../0_stateless/01114_database_atomic.sh | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index fed76727a27..f74036c8462 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -53,10 +53,20 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UIn $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +RANDOM_COMMENT="$RANDOM" +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) -sleep 1 # SELECT and INSERT should start before the following RENAMEs +it=0 +while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do + it=$((it+1)) + if [ $it -ge 50 ]; + then + echo "Failed to wait for first batch of queries" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'" + fi + sleep 0.1 +done $CLICKHOUSE_CLIENT -nm -q " RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp; @@ -79,8 +89,18 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) -sleep 1 +RANDOM_TUPLE="${RANDOM}_tuple" +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +it=0 +while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do + it=$((it+1)) + if [ $it -ge 50 ]; + then + echo "Failed to wait for second batch of queries" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'" + fi + sleep 0.1 +done $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" wait # for INSERT and SELECT From 2d2e8b3a967b5455870d6a9dd873c1deeaf77af3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 11:52:24 +0200 Subject: [PATCH 371/409] Make the test parallelizable --- .../01114_database_atomic.reference | 17 ++-- .../0_stateless/01114_database_atomic.sh | 89 +++++++++---------- 2 files changed, 50 insertions(+), 56 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index f42cd099d4e..33080a7a77b 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -1,17 +1,14 @@ 1 -CREATE DATABASE test_01114_1\nENGINE = Atomic -CREATE DATABASE test_01114_2\nENGINE = Atomic -CREATE DATABASE test_01114_3\nENGINE = Ordinary -test_01114_1 Atomic store 00001114-1000-4000-8000-000000000001 1 -test_01114_2 Atomic store 00001114-1000-4000-8000-000000000002 1 -test_01114_3 Ordinary test_01114_3 test_01114_3 1 +CREATE DATABASE default_1\nENGINE = Atomic +CREATE DATABASE default_2\nENGINE = Atomic +CREATE DATABASE default_3\nENGINE = Ordinary 110 100 -CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE default_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 110 -CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 5 dropped 110 5995 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index f74036c8462..412f5336224 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -9,25 +9,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +DATABASE_1="${CLICKHOUSE_DATABASE}_1" +DATABASE_2="${CLICKHOUSE_DATABASE}_2" +DATABASE_3="${CLICKHOUSE_DATABASE}_3" -$CLICKHOUSE_CLIENT -nm -q " -DROP DATABASE IF EXISTS test_01114_1; -DROP DATABASE IF EXISTS test_01114_2; -DROP DATABASE IF EXISTS test_01114_3; -" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE ${DATABASE_1} ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" -$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_1} ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_2}" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE ${DATABASE_3} ENGINE=Ordinary" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_2" -$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01114_3 ENGINE=Ordinary" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_1}" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_2}" +$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE ${DATABASE_3}" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_2" -$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_3" - -uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_1'"` -uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_2'"` +uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_1}'"` +uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_2}'"` $CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], @@ -35,27 +32,27 @@ $CLICKHOUSE_CLIENT -q "SELECT name, FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " -CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); -INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100); -CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5); -INSERT INTO test_01114_3.mt SELECT * FROM numbers(110); +CREATE TABLE ${DATABASE_1}.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO ${DATABASE_1}.mt_tmp SELECT * FROM numbers(100); +CREATE TABLE ${DATABASE_3}.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5); +INSERT INTO ${DATABASE_3}.mt SELECT * FROM numbers(110); -RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp; /* move from Atomic to Ordinary */ -RENAME TABLE test_01114_3.mt TO test_01114_1.mt; /* move from Ordinary to Atomic */ -SELECT count() FROM test_01114_1.mt; -SELECT count() FROM test_01114_3.mt_tmp; +RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_3}.mt_tmp; /* move from Atomic to Ordinary */ +RENAME TABLE ${DATABASE_3}.mt TO ${DATABASE_1}.mt; /* move from Ordinary to Atomic */ +SELECT count() FROM ${DATABASE_1}.mt; +SELECT count() FROM ${DATABASE_3}.mt_tmp; -DROP DATABASE test_01114_3; +DROP DATABASE ${DATABASE_3}; " explicit_uuid=$($CLICKHOUSE_CLIENT -q "SELECT generateUUIDv4()") -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${DATABASE_2}.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='${DATABASE_2}'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" RANDOM_COMMENT="$RANDOM" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do @@ -69,28 +66,28 @@ while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE que done $CLICKHOUSE_CLIENT -nm -q " -RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp; -RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp; -EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp; -RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt; -EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt; +RENAME TABLE ${DATABASE_1}.mt TO ${DATABASE_1}.mt_tmp; +RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_2}.mt_tmp; +EXCHANGE TABLES ${DATABASE_2}.mt AND ${DATABASE_2}.mt_tmp; +RENAME TABLE ${DATABASE_2}.mt_tmp TO ${DATABASE_1}.mt; +EXCHANGE TABLES ${DATABASE_1}.mt AND ${DATABASE_2}.mt; " # Check that nothing changed -$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" -uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='test_01114_1' AND name='mt'") -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM ${DATABASE_1}.mt" +uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='${DATABASE_1}' AND name='mt'") +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_1}.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " -DROP TABLE test_01114_1.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0; -CREATE TABLE test_01114_1.mt (s String) ENGINE=Log(); -INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); -SELECT count() FROM test_01114_1.mt +DROP TABLE ${DATABASE_1}.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0; +CREATE TABLE ${DATABASE_1}.mt (s String) ENGINE=Log(); +INSERT INTO ${DATABASE_1}.mt SELECT 's' || toString(number) FROM numbers(5); +SELECT count() FROM ${DATABASE_1}.mt " # result: 5 RANDOM_TUPLE="${RANDOM}_tuple" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do it=$((it+1)) @@ -101,9 +98,9 @@ while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE que fi sleep 0.1 done -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_1}" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" wait # for INSERT and SELECT -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435 -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${DATABASE_2}.mt" # result: 30, 435 +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_2}" --database_atomic_wait_for_drop_and_detach_synchronously=0 From 94f168e5ef6fb1e7121197cef9bd32de0ff391d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 27 Aug 2024 12:12:34 +0200 Subject: [PATCH 372/409] CI: Fix job rerun check --- tests/ci/ci_utils.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index e8d9e7dc254..cd7a271f892 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -286,4 +286,6 @@ class Utils: @staticmethod def is_job_triggered_manually(): - return "robot" not in Envs.GITHUB_ACTOR + return ( + "robot" not in Envs.GITHUB_ACTOR and "clickhouse-ci" not in Envs.GITHUB_ACTOR + ) From 0ad66778da65f9e6c2659044343763e47f3f4878 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 27 Aug 2024 10:19:19 +0000 Subject: [PATCH 373/409] Automatic style fix --- tests/ci/ci_utils.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index cd7a271f892..8b60f61b006 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -287,5 +287,6 @@ class Utils: @staticmethod def is_job_triggered_manually(): return ( - "robot" not in Envs.GITHUB_ACTOR and "clickhouse-ci" not in Envs.GITHUB_ACTOR + "robot" not in Envs.GITHUB_ACTOR + and "clickhouse-ci" not in Envs.GITHUB_ACTOR ) From 43482c4ed0cec53680314357ba2a91891b27dd73 Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Tue, 27 Aug 2024 10:42:47 +0000 Subject: [PATCH 374/409] Recreate database at each run --- .../test_distributed.py | 37 +++++++++---------- .../test_truncate_database/test_replicated.py | 5 ++- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_truncate_database/test_distributed.py b/tests/integration/test_truncate_database/test_distributed.py index fed3d16d190..67dea70b1f4 100644 --- a/tests/integration/test_truncate_database/test_distributed.py +++ b/tests/integration/test_truncate_database/test_distributed.py @@ -1,40 +1,35 @@ import pytest - from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/distributed_servers.xml"], with_zookeeper=True + "node1", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) node2 = cluster.add_instance( "node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) +def fill_nodes(nodes): + for node in nodes: + node.query( + """ + DROP DATABASE IF EXISTS test; + CREATE DATABASE test; + CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id; + CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table); + INSERT INTO test.local_table VALUES ({pos}, '{replica}'); + """.format( + pos=node.name[4:], replica=node.name + ) + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - for node in (node1, node2): - node.query( - """ - CREATE DATABASE test; - CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id - """ - ) - - node1.query("INSERT INTO test.local_table VALUES (1, 'node1')") - node2.query("INSERT INTO test.local_table VALUES (2, 'node2')") - - node1.query( - "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" - ) - node2.query( - "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" - ) - yield cluster finally: @@ -42,6 +37,8 @@ def started_cluster(): def test_truncate_database_distributed(started_cluster): + fill_nodes([node1, node2]) + query1 = "SELECT count() FROM test.distributed WHERE (id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))" query2 = "SELECT sum((id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))) FROM test.distributed" assert node1.query(query1) == "1\n" diff --git a/tests/integration/test_truncate_database/test_replicated.py b/tests/integration/test_truncate_database/test_replicated.py index d55a0dc7557..edcc0446da3 100644 --- a/tests/integration/test_truncate_database/test_replicated.py +++ b/tests/integration/test_truncate_database/test_replicated.py @@ -6,6 +6,7 @@ def fill_nodes(nodes, shard): for node in nodes: node.query( """ + DROP DATABASE IF EXISTS test; CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) @@ -33,8 +34,6 @@ def start_cluster(): try: cluster.start() - fill_nodes([node1, node2, node3], 1) - yield cluster except Exception as ex: @@ -45,6 +44,8 @@ def start_cluster(): def test_truncate_database_replicated(start_cluster): + fill_nodes([node1, node2, node3], 1) + node1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) From c7d0d790e2b37bcd91f5e147d775e656bf3d22a7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Aug 2024 12:38:37 +0000 Subject: [PATCH 375/409] fix materialized views with optimize_functions_to_subcolumns --- .../Passes/FunctionToSubcolumnsPass.cpp | 10 +++-- .../0_stateless/03230_subcolumns_mv.reference | 1 + .../0_stateless/03230_subcolumns_mv.sql | 37 +++++++++++++++++++ 3 files changed, 45 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03230_subcolumns_mv.reference create mode 100644 tests/queries/0_stateless/03230_subcolumns_mv.sql diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 1fc3eec6833..6caf69e3a2c 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -209,7 +209,7 @@ std::map, NodeToSubcolumnTransformer> node_transfor }, }; -std::tuple getTypedNodesForOptimization(const QueryTreeNodePtr & node) +std::tuple getTypedNodesForOptimization(const QueryTreeNodePtr & node, const ContextPtr & context) { auto * function_node = node->as(); if (!function_node) @@ -232,6 +232,10 @@ std::tuple getTypedNodesForOptimizati const auto & storage_snapshot = table_node->getStorageSnapshot(); auto column = first_argument_column_node->getColumn(); + auto view_source = context->getViewSource(); + if (view_source && view_source->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted()) + return {}; + if (!storage->supportsOptimizationToSubcolumns() || storage->isVirtualColumn(column.name, storage_snapshot->metadata)) return {}; @@ -266,7 +270,7 @@ public: return; } - auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node); + auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node, getContext()); if (function_node && first_argument_node && table_node) { enterImpl(*function_node, *first_argument_node, *table_node); @@ -416,7 +420,7 @@ public: if (!getSettings().optimize_functions_to_subcolumns) return; - auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node); + auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node, getContext()); if (!function_node || !first_argument_column_node || !table_node) return; diff --git a/tests/queries/0_stateless/03230_subcolumns_mv.reference b/tests/queries/0_stateless/03230_subcolumns_mv.reference new file mode 100644 index 00000000000..03528148b49 --- /dev/null +++ b/tests/queries/0_stateless/03230_subcolumns_mv.reference @@ -0,0 +1 @@ +['key1','key2'] ['value1','value2'] diff --git a/tests/queries/0_stateless/03230_subcolumns_mv.sql b/tests/queries/0_stateless/03230_subcolumns_mv.sql new file mode 100644 index 00000000000..e2e577f54c1 --- /dev/null +++ b/tests/queries/0_stateless/03230_subcolumns_mv.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS rawtable; +DROP TABLE IF EXISTS raw_to_attributes_mv; +DROP TABLE IF EXISTS attributes; + +SET optimize_functions_to_subcolumns = 1; + +CREATE TABLE rawtable +( + `Attributes` Map(String, String), +) +ENGINE = MergeTree +ORDER BY tuple(); + +CREATE MATERIALIZED VIEW raw_to_attributes_mv TO attributes +( + `AttributeKeys` Array(String), + `AttributeValues` Array(String) +) +AS SELECT + mapKeys(Attributes) AS AttributeKeys, + mapValues(Attributes) AS AttributeValues +FROM rawtable; + +CREATE TABLE attributes +( + `AttributeKeys` Array(String), + `AttributeValues` Array(String) +) +ENGINE = ReplacingMergeTree +ORDER BY tuple(); + +INSERT INTO rawtable VALUES ({'key1': 'value1', 'key2': 'value2'}); +SELECT * FROM raw_to_attributes_mv ORDER BY AttributeKeys; + +DROP TABLE IF EXISTS rawtable; +DROP TABLE IF EXISTS raw_to_attributes_mv; +DROP TABLE IF EXISTS attributes; From 6ee197ea59d9f7f5d3e8dc24b8eded763c65d024 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Aug 2024 13:01:37 +0000 Subject: [PATCH 376/409] Don't use serializations cache in const Dynamic column methods --- src/Columns/ColumnDynamic.cpp | 11 +++++------ src/Columns/ColumnDynamic.h | 6 +++--- src/DataTypes/DataTypeDynamic.cpp | 2 +- src/DataTypes/Serializations/SerializationDynamic.cpp | 5 ++--- ...namic_serializations_uninitialized_value.reference | 4 ++++ ...228_dynamic_serializations_uninitialized_value.sql | 4 ++++ 6 files changed, 19 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference create mode 100644 tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..a66db43d6d4 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -300,7 +300,7 @@ void ColumnDynamic::get(size_t n, Field & res) const auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n)); ReadBufferFromMemory buf(value_data.data, value_data.size); auto type = decodeDataType(buf); - getVariantSerialization(type)->deserializeBinary(res, buf, getFormatSettings()); + type->getDefaultSerialization()->deserializeBinary(res, buf, getFormatSettings()); } @@ -736,8 +736,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const { const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); encodeDataType(variant_type, buf); - getVariantSerialization(variant_type, variant_info.variant_names[discr]) - ->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); + variant_type->getDefaultSerialization()->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); type_and_value = buf.str(); } @@ -870,7 +869,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have both values serialized in binary format, so we need to /// create temporary column, insert both values into it and compare. auto tmp_column = left_data_type->createColumn(); - const auto & serialization = getVariantSerialization(left_data_type, left_data_type_name); + const auto & serialization = left_data_type->getDefaultSerialization(); serialization->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); serialization->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); return tmp_column->compareAt(0, 1, *tmp_column, nan_direction_hint); @@ -892,7 +891,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have left value serialized in binary format, we need to /// create temporary column, insert the value into it and compare. auto tmp_column = left_data_type->createColumn(); - getVariantSerialization(left_data_type, left_data_type_name)->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + left_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); return tmp_column->compareAt(0, right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); } /// Check if only right value is in shared data. @@ -912,7 +911,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have right value serialized in binary format, we need to /// create temporary column, insert the value into it and compare. auto tmp_column = right_data_type->createColumn(); - getVariantSerialization(right_data_type, right_data_type_name)->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + right_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), 0, *tmp_column, nan_direction_hint); } /// Otherwise both values are regular variants. diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..72542a15530 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -414,7 +414,7 @@ public: /// Insert value into shared variant. Also updates Variant discriminators and offsets. void insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n); - const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) const + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) { /// Get serialization for provided data type. /// To avoid calling type->getDefaultSerialization() every time we use simple cache with max size. @@ -428,7 +428,7 @@ public: return serialization_cache.emplace(variant_name, variant_type->getDefaultSerialization()).first->second; } - const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) { return getVariantSerialization(variant_type, variant_type->getName()); } private: void createVariantInfo(const DataTypePtr & variant_type); @@ -473,7 +473,7 @@ private: /// We can use serializations of different data types to serialize values into shared variant. /// To avoid creating the same serialization multiple times, use simple cache. static const size_t SERIALIZATION_CACHE_MAX_SIZE = 256; - mutable std::unordered_map serialization_cache; + std::unordered_map serialization_cache; }; void extendVariantColumn( diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index c35f7526a18..fb938f5fbd8 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -185,7 +185,7 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa auto type = decodeDataType(buf); if (type->getName() == subcolumn_type_name) { - dynamic_column.getVariantSerialization(subcolumn_type, subcolumn_type_name)->deserializeBinary(*subcolumn, buf, format_settings); + subcolumn_type->getDefaultSerialization()->deserializeBinary(*subcolumn, buf, format_settings); null_map.push_back(0); } else diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index b921a3bc897..32964e17bce 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -489,9 +489,8 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); - const auto & variant_type_name = variant_info.variant_names[global_discr]; encodeDataType(variant_type, ostr); - dynamic_column.getVariantSerialization(variant_type, variant_type_name)->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); + variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } template @@ -629,7 +628,7 @@ static void serializeTextImpl( ReadBufferFromMemory buf(value.data, value.size); auto variant_type = decodeDataType(buf); auto tmp_variant_column = variant_type->createColumn(); - auto variant_serialization = dynamic_column.getVariantSerialization(variant_type); + auto variant_serialization = variant_type->getDefaultSerialization(); variant_serialization->deserializeBinary(*tmp_variant_column, buf, settings); nested_serialize(*variant_serialization, *tmp_variant_column, 0, ostr); } diff --git a/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference new file mode 100644 index 00000000000..64c193a8499 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference @@ -0,0 +1,4 @@ +str 3 \N +str 3 \N +str 3 \N +str 3 \N diff --git a/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql new file mode 100644 index 00000000000..8a565fe36b9 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql @@ -0,0 +1,4 @@ +set allow_experimental_dynamic_type=1; +set cast_keep_nullable=1; +SELECT toFixedString('str', 3), 3, CAST(if(1 = 0, toInt8(3), NULL), 'Int32') AS x from numbers(10) GROUP BY GROUPING SETS ((CAST(toInt32(1), 'Int32')), ('str', 3), (CAST(toFixedString('str', 3), 'Dynamic')), (CAST(toFixedString(toFixedString('str', 3), 3), 'Dynamic'))); + From 88b72238c860b654657bfe56a10f4b864571bfb2 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Aug 2024 13:09:19 +0000 Subject: [PATCH 377/409] Fix JSON type parsing with nulls as defaults --- src/Formats/JSONExtractTree.cpp | 20 +++++++++++++++++++ ...9_json_null_as_default_for_tuple.reference | 1 + .../03229_json_null_as_default_for_tuple.sql | 4 ++++ 3 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference create mode 100644 tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 122224535a7..9ea335ee7fe 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1179,6 +1179,12 @@ public: const FormatSettings & format_settings, String & error) const override { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + auto & tuple = assert_cast(column); size_t old_size = column.size(); bool were_valid_elements = false; @@ -1298,6 +1304,12 @@ public: const FormatSettings & format_settings, String & error) const override { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + if (!element.isObject()) { error = fmt::format("cannot read Map value from JSON element: {}", jsonElementToString(element, format_settings)); @@ -1362,6 +1374,14 @@ public: String & error) const override { auto & column_variant = assert_cast(column); + + /// Check if element is NULL. + if (element.isNull()) + { + column_variant.insertDefault(); + return true; + } + for (size_t i : order) { auto & variant = column_variant.getVariantByGlobalDiscriminator(i); diff --git a/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference new file mode 100644 index 00000000000..fbbba67a2b8 --- /dev/null +++ b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference @@ -0,0 +1 @@ +{"a":[["1",{}],[null,{}]]} [(1,'{}'),(NULL,'{}')] Array(Tuple(Nullable(Int64), JSON(max_dynamic_types=16, max_dynamic_paths=256))) diff --git a/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql new file mode 100644 index 00000000000..d405f6232e4 --- /dev/null +++ b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql @@ -0,0 +1,4 @@ +set allow_experimental_json_type=1; + +select materialize('{"a" : [[1, {}], null]}')::JSON as json, getSubcolumn(json, 'a'), dynamicType(getSubcolumn(json, 'a')); + From 35adf8d12b7ff906a1b846f9547aea792041d476 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 27 Aug 2024 15:22:54 +0200 Subject: [PATCH 378/409] add toBool to docs --- .../functions/type-conversion-functions.md | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index edd04580f27..f7fd2d68cf7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -49,6 +49,55 @@ SETTINGS cast_keep_nullable = 1 └──────────────────┴─────────────────────┴──────────────────┘ ``` +## toBool + +Converts an input value to a value of type [`Bool`](../data-types/boolean.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toBool(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- Strings `true` or `false` (case-insensitive). + +**Returned value** + +- Returns `true` or `false` based on evaluation of the argument. [Bool](../data-types/boolean.md). + +**Example** + +Query: + +```sql +SELECT + toBool(toUInt8(1)), + toBool(toInt8(-1)), + toBool(toFloat32(1.01)), + toBool('true'), + toBool('false'), + toBool('FALSE') +FORMAT Vertical +``` + +Result: + +```response +toBool(toUInt8(1)): true +toBool(toInt8(-1)): true +toBool(toFloat32(1.01)): true +toBool('true'): true +toBool('false'): false +toBool('FALSE'): false +``` + ## toInt8 Converts an input value to a value of type [`Int8`](../data-types/int-uint.md). Throws an exception in case of an error. From c884a2fb9d15604cbb82f4b41636fa4d15ebb175 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Aug 2024 13:26:20 +0000 Subject: [PATCH 379/409] perf tests set cgroups_memory_usage_observer_wait_time to zero --- docker/test/performance-comparison/run.sh | 5 +++-- .../config.d/cgroups_memory_usage_observer_wait_time.xml | 3 +++ tests/performance/scripts/entrypoint.sh | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) create mode 100644 tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml diff --git a/docker/test/performance-comparison/run.sh b/docker/test/performance-comparison/run.sh index 6ef781fa4c8..e40f643e08c 100644 --- a/docker/test/performance-comparison/run.sh +++ b/docker/test/performance-comparison/run.sh @@ -13,7 +13,8 @@ entry="/usr/share/clickhouse-test/performance/scripts/entrypoint.sh" # https://www.kernel.org/doc/Documentation/filesystems/tmpfs.txt # Double-escaped backslashes are a tribute to the engineering wonder of docker -- # it gives '/bin/sh: 1: [bash,: not found' otherwise. -numactl --hardware +echo > compare.log +numactl --hardware | tee -a compare.log node=$(( RANDOM % $(numactl --hardware | sed -n 's/^.*available:\(.*\)nodes.*$/\1/p') )); -echo Will bind to NUMA node $node; +echo Will bind to NUMA node $node | tee -a compare.log numactl --cpunodebind=$node --membind=$node $entry diff --git a/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml b/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml new file mode 100644 index 00000000000..a7868b11212 --- /dev/null +++ b/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/performance/scripts/entrypoint.sh b/tests/performance/scripts/entrypoint.sh index db7d96ad150..d392ba7c410 100755 --- a/tests/performance/scripts/entrypoint.sh +++ b/tests/performance/scripts/entrypoint.sh @@ -152,7 +152,7 @@ cat /proc/sys/kernel/core_pattern { time $SCRIPT_DIR/download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ time stage=configure "$script_path"/compare.sh ; \ -} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee compare.log +} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee -a compare.log # Stop the servers to free memory. Normally they are restarted before getting # the profile info, so they shouldn't use much, but if the comparison script From 53bfab4703cd6fd2369c0ad6aa8fe50e1fd77e78 Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Tue, 27 Aug 2024 14:20:31 +0000 Subject: [PATCH 380/409] Missing newline --- tests/integration/test_truncate_database/test_distributed.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_truncate_database/test_distributed.py b/tests/integration/test_truncate_database/test_distributed.py index 67dea70b1f4..6b822077a3c 100644 --- a/tests/integration/test_truncate_database/test_distributed.py +++ b/tests/integration/test_truncate_database/test_distributed.py @@ -10,6 +10,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) + def fill_nodes(nodes): for node in nodes: node.query( From 90cc6199664705c0c0214f60b4cbb246480d372d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 15:06:43 +0000 Subject: [PATCH 381/409] Better care about grouping keys order for GROUPING SETS. --- src/Interpreters/Aggregator.h | 12 + src/Interpreters/InterpreterSelectQuery.cpp | 51 ++-- src/Planner/Planner.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 101 ++++--- src/Processors/QueryPlan/AggregatingStep.h | 19 +- .../QueryPlan/MergingAggregatedStep.cpp | 14 +- .../QueryPlan/MergingAggregatedStep.h | 2 + .../Transforms/MergingAggregatedTransform.cpp | 196 ++++++++++--- .../Transforms/MergingAggregatedTransform.h | 26 +- .../02165_replicated_grouping_sets.reference | 266 ++++++++++++++---- .../02165_replicated_grouping_sets.sql | 23 +- 11 files changed, 517 insertions(+), 194 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index f4f1e9a1df3..2cb04fc7c51 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -59,6 +59,18 @@ class CompiledAggregateFunctionsHolder; class NativeWriter; struct OutputBlockColumns; +struct GroupingSetsParams +{ + GroupingSetsParams() = default; + + GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { } + + Names used_keys; + Names missing_keys; +}; + +using GroupingSetsParamsList = std::vector; + /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingTransform.) * diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9e5fffac6e4..ca0e84a5267 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -347,6 +347,27 @@ bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) return false; } +GroupingSetsParamsList getAggregatorGroupingSetsParams(const NamesAndTypesLists & aggregation_keys_list, const Names & all_keys) +{ + GroupingSetsParamsList result; + + for (const auto & aggregation_keys : aggregation_keys_list) + { + NameSet keys; + for (const auto & key : aggregation_keys) + keys.insert(key.name); + + Names missing_keys; + for (const auto & key : all_keys) + if (!keys.contains(key)) + missing_keys.push_back(key); + + result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys)); + } + + return result; +} + } InterpreterSelectQuery::InterpreterSelectQuery( @@ -2005,6 +2026,7 @@ static void executeMergeAggregatedImpl( bool has_grouping_sets, const Settings & settings, const NamesAndTypesList & aggregation_keys, + const NamesAndTypesLists & aggregation_keys_list, const AggregateDescriptions & aggregates, bool should_produce_results_in_order_of_bucket_number, SortDescription group_by_sort_description) @@ -2027,10 +2049,12 @@ static void executeMergeAggregatedImpl( */ Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size, settings.min_hit_rate_to_use_consecutive_keys_optimization); + auto grouping_sets_params = getAggregatorGroupingSetsParams(aggregation_keys_list, keys); auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), params, + grouping_sets_params, final, /// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989) settings.distributed_aggregation_memory_efficient && is_remote_storage && !has_grouping_sets, @@ -2651,30 +2675,6 @@ static Aggregator::Params getAggregatorParams( }; } -static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryExpressionAnalyzer & query_analyzer, const Names & all_keys) -{ - GroupingSetsParamsList result; - if (query_analyzer.useGroupingSetKey()) - { - auto const & aggregation_keys_list = query_analyzer.aggregationKeysList(); - - for (const auto & aggregation_keys : aggregation_keys_list) - { - NameSet keys; - for (const auto & key : aggregation_keys) - keys.insert(key.name); - - Names missing_keys; - for (const auto & key : all_keys) - if (!keys.contains(key)) - missing_keys.push_back(key); - - result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys)); - } - } - return result; -} - void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { executeExpression(query_plan, expression, "Before GROUP BY"); @@ -2694,7 +2694,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes); - auto grouping_sets_params = getAggregatorGroupingSetsParams(*query_analyzer, keys); + auto grouping_sets_params = getAggregatorGroupingSetsParams(query_analyzer->aggregationKeysList(), keys); SortDescription group_by_sort_description; SortDescription sort_description_for_merging; @@ -2762,6 +2762,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool has_grouping_sets, context->getSettingsRef(), query_analyzer->aggregationKeys(), + query_analyzer->aggregationKeysList(), query_analyzer->aggregates(), should_produce_results_in_order_of_bucket_number, std::move(group_by_sort_description)); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c0efed8550f..7b5101c5c7d 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -528,6 +528,7 @@ void addMergingAggregatedStep(QueryPlan & query_plan, auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), params, + aggregation_analysis_result.grouping_sets_parameters_list, query_analysis_result.aggregate_final, /// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989) settings.distributed_aggregation_memory_efficient && (is_remote_storage || parallel_replicas_from_merge_tree) && !query_analysis_result.aggregation_with_rollup_or_cube_or_grouping_sets, diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 8a5ed7fde65..a4d707704b1 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -151,6 +151,61 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_, explicit_sorting_required_for_aggregation_in_order = false; } +ActionsDAG AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( + const Block & in_header, + const Block & out_header, + const GroupingSetsParamsList & grouping_sets_params, + UInt64 group, + bool group_by_use_nulls) +{ + /// Here we create a DAG which fills missing keys and adds `__grouping_set` column + ActionsDAG dag(in_header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs outputs; + outputs.reserve(out_header.columns() + 1); + + auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, group), 0); + const auto * grouping_node = &dag.addColumn( + {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); + + grouping_node = &dag.materializeNode(*grouping_node); + outputs.push_back(grouping_node); + + const auto & missing_columns = grouping_sets_params[group].missing_keys; + const auto & used_keys = grouping_sets_params[group].used_keys; + + auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); + for (size_t i = 0; i < out_header.columns(); ++i) + { + const auto & col = out_header.getByPosition(i); + const auto missing_it = std::find_if( + missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; }); + const auto used_it = std::find_if( + used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; }); + if (missing_it != missing_columns.end()) + { + auto column_with_default = col.column->cloneEmpty(); + col.type->insertDefaultInto(*column_with_default); + column_with_default->finalize(); + + auto column = ColumnConst::create(std::move(column_with_default), 0); + const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); + node = &dag.materializeNode(*node); + outputs.push_back(node); + } + else + { + const auto * column_node = dag.getOutputs()[in_header.getPositionByName(col.name)]; + if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) + outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); + else + outputs.push_back(column_node); + } + } + + dag.getOutputs().swap(outputs); + return dag; +} + void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { QueryPipelineProcessorsCollector collector(pipeline, this); @@ -300,51 +355,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { const auto & header = ports[set_counter]->getHeader(); - /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - ActionsDAG dag(header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs outputs; - outputs.reserve(output_header.columns() + 1); - - auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0); - const auto * grouping_node = &dag.addColumn( - {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - - grouping_node = &dag.materializeNode(*grouping_node); - outputs.push_back(grouping_node); - - const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; - const auto & used_keys = grouping_sets_params[set_counter].used_keys; - - auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); - for (size_t i = 0; i < output_header.columns(); ++i) - { - auto & col = output_header.getByPosition(i); - const auto missing_it = std::find_if( - missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; }); - const auto used_it = std::find_if( - used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; }); - if (missing_it != missing_columns.end()) - { - auto column_with_default = col.column->cloneEmpty(); - col.type->insertDefaultInto(*column_with_default); - column_with_default->finalize(); - - auto column = ColumnConst::create(std::move(column_with_default), 0); - const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); - node = &dag.materializeNode(*node); - outputs.push_back(node); - } - else - { - const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)]; - if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) - outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); - else - outputs.push_back(column_node); - } - } - - dag.getOutputs().swap(outputs); + auto dag = makeCreatingMissingKeysForGroupingSetDAG(header, output_header, grouping_sets_params, set_counter, group_by_use_nulls); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index ae43295024a..4e4078047f1 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -7,18 +7,6 @@ namespace DB { -struct GroupingSetsParams -{ - GroupingSetsParams() = default; - - GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { } - - Names used_keys; - Names missing_keys; -}; - -using GroupingSetsParamsList = std::vector; - Block appendGroupingSetColumn(Block header); Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls); @@ -77,6 +65,13 @@ public: /// Argument input_stream would be the second input (from projection). std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; + static ActionsDAG makeCreatingMissingKeysForGroupingSetDAG( + const Block & in_header, + const Block & out_header, + const GroupingSetsParamsList & grouping_sets_params, + UInt64 group, + bool group_by_use_nulls); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 7207b5e6c7f..f3eb352faac 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -42,6 +42,7 @@ static ITransformingStep::Traits getTraits(bool should_produce_results_in_order_ MergingAggregatedStep::MergingAggregatedStep( const DataStream & input_stream_, Aggregator::Params params_, + GroupingSetsParamsList grouping_sets_params_, bool final_, bool memory_efficient_aggregation_, size_t max_threads_, @@ -56,6 +57,7 @@ MergingAggregatedStep::MergingAggregatedStep( MergingAggregatedTransform::appendGroupingIfNeeded(input_stream_.header, params_.getHeader(input_stream_.header, final_)), getTraits(should_produce_results_in_order_of_bucket_number_)) , params(std::move(params_)) + , grouping_sets_params(std::move(grouping_sets_params_)) , final(final_) , memory_efficient_aggregation(memory_efficient_aggregation_) , max_threads(max_threads_) @@ -94,14 +96,13 @@ void MergingAggregatedStep::applyOrder(SortDescription sort_description, DataStr void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); - if (memoryBoundMergingWillBeUsed()) { - if (input_streams.front().header.has("__grouping_set")) + if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Memory bound merging of aggregated results is not supported for grouping sets."); + auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), @@ -136,18 +137,19 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c pipeline.resize(1); /// Now merge the aggregated blocks - pipeline.addSimpleTransform([&](const Block & header) - { return std::make_shared(header, transform_params, max_threads); }); + auto transform = std::make_shared(pipeline.getHeader(), params, final, grouping_sets_params, max_threads); + pipeline.addTransform(std::move(transform)); } else { - if (input_streams.front().header.has("__grouping_set")) + if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Memory efficient merging of aggregated results is not supported for grouping sets."); auto num_merge_threads = memory_efficient_merge_threads ? memory_efficient_merge_threads : max_threads; + auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); pipeline.addMergingAggregatedMemoryEfficientTransform(transform_params, num_merge_threads); } diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 654f794d5f5..5c3842a6c33 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -16,6 +16,7 @@ public: MergingAggregatedStep( const DataStream & input_stream_, Aggregator::Params params_, + GroupingSetsParamsList grouping_sets_params_, bool final_, bool memory_efficient_aggregation_, size_t max_threads_, @@ -43,6 +44,7 @@ private: Aggregator::Params params; + GroupingSetsParamsList grouping_sets_params; bool final; bool memory_efficient_aggregation; size_t max_threads; diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 78fb2f340bf..cf383cfcf9d 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -1,7 +1,9 @@ #include #include #include +#include #include +#include #include namespace DB @@ -23,19 +25,93 @@ Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header return out_header; } +MergingAggregatedTransform::~MergingAggregatedTransform() = default; + MergingAggregatedTransform::MergingAggregatedTransform( - Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_) - : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params_->getHeader())) - , params(std::move(params_)), max_threads(max_threads_), has_grouping_sets(header_.has("__grouping_set")) + Block header_, + Aggregator::Params params, + bool final, + GroupingSetsParamsList grouping_sets_params, + size_t max_threads_) + : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params.getHeader(header_, final))) + , max_threads(max_threads_) { + if (!grouping_sets_params.empty()) + { + if (!header_.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot find __grouping_set column in header of MergingAggregatedTransform with grouping sets." + "Header {}", header_.dumpStructure()); + + auto in_header = header_; + in_header.erase(header_.getPositionByName("__grouping_set")); + auto out_header = params.getHeader(header_, final); + + grouping_sets.reserve(grouping_sets_params.size()); + for (const auto & grouping_set_params : grouping_sets_params) + { + size_t group = grouping_sets.size(); + + ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); + auto & outputs = reordering.getOutputs(); + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(in_header.columns() + grouping_set_params.used_keys.size() - grouping_set_params.used_keys.size()); + + std::unordered_map index; + for (size_t pos = 0; pos < outputs.size(); ++pos) + index.emplace(outputs[pos]->result_name, pos); + + for (const auto & used_name : grouping_set_params.used_keys) + { + auto & idx = index[used_name]; + new_outputs.push_back(outputs[idx]); + } + + for (const auto & used_name : grouping_set_params.used_keys) + index[used_name] = outputs.size(); + for (const auto & missing_name : grouping_set_params.missing_keys) + index[missing_name] = outputs.size(); + + for (const auto * output : outputs) + { + if (index[output->result_name] != outputs.size()) + new_outputs.push_back(output); + } + + outputs.swap(new_outputs); + + Aggregator::Params set_params(grouping_set_params.used_keys, + params.aggregates, + params.overflow_row, + params.max_threads, + params.max_block_size, + params.min_hit_rate_to_use_consecutive_keys_optimization); + + auto transform_params = std::make_shared(reordering.updateHeader(in_header), std::move(set_params), final); + + auto creating = AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( + transform_params->getHeader(), + out_header, + grouping_sets_params, group, false); + + auto & groupiung_set = grouping_sets.emplace_back(); + groupiung_set.reordering_key_columns_actions = std::make_shared(std::move(reordering)); + groupiung_set.creating_missing_keys_actions = std::make_shared(std::move(creating)); + groupiung_set.params = std::move(transform_params); + } + } + else + { + auto & groupiung_set = grouping_sets.emplace_back(); + groupiung_set.params = std::make_shared(header_, std::move(params), final); + } } void MergingAggregatedTransform::addBlock(Block block) { - if (!has_grouping_sets) + if (grouping_sets.size() == 1) { - auto & bucket_to_blocks = grouping_sets[0]; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + grouping_sets[0].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); return; } @@ -49,13 +125,12 @@ void MergingAggregatedTransform::addBlock(Block block) if (!grouping_column_typed) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); - /// Enumerate groups and fill the selector. - std::map enumerated_groups; IColumn::Selector selector; const auto & grouping_data = grouping_column_typed->getData(); size_t num_rows = grouping_data.size(); UInt64 last_group = grouping_data[0]; + UInt64 max_group = last_group; for (size_t row = 1; row < num_rows; ++row) { auto group = grouping_data[row]; @@ -65,32 +140,32 @@ void MergingAggregatedTransform::addBlock(Block block) continue; /// Optimization for single group. - if (enumerated_groups.empty()) - { + if (selector.empty()) selector.reserve(num_rows); - enumerated_groups.emplace(last_group, enumerated_groups.size()); - } /// Fill the last equal range. - selector.resize_fill(row, enumerated_groups[last_group]); - /// Enumerate new group if did not see it before. - enumerated_groups.emplace(group, enumerated_groups.size()); - + selector.resize_fill(row, last_group); last_group = group; + max_group = std::max(last_group, max_group); } + if (max_group >= grouping_sets.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid group number {}. Number of groups {}.", last_group, grouping_sets.size()); + /// Optimization for single group. - if (enumerated_groups.empty()) + if (selector.empty()) { - auto & bucket_to_blocks = grouping_sets[last_group]; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + auto bucket = block.info.bucket_num; + grouping_sets[last_group].reordering_key_columns_actions->execute(block); + grouping_sets[last_group].bucket_to_blocks[bucket].emplace_back(std::move(block)); return; } /// Fill the last equal range. - selector.resize_fill(num_rows, enumerated_groups[last_group]); + selector.resize_fill(num_rows, last_group); - const size_t num_groups = enumerated_groups.size(); + const size_t num_groups = max_group + 1; Blocks splitted_blocks(num_groups); for (size_t group_id = 0; group_id < num_groups; ++group_id) @@ -104,28 +179,28 @@ void MergingAggregatedTransform::addBlock(Block block) splitted_blocks[group_id].getByPosition(col_idx_in_block).column = std::move(splitted_columns[group_id]); } - for (auto [group, group_id] : enumerated_groups) + for (size_t group = 0; group < num_groups; ++group) { - auto & bucket_to_blocks = grouping_sets[group]; - auto & splitted_block = splitted_blocks[group_id]; + auto & splitted_block = splitted_blocks[group]; splitted_block.info = block.info; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); + grouping_sets[group].reordering_key_columns_actions->execute(splitted_block); + grouping_sets[group].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); } } -void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) -{ - auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); - for (auto & block : block_list) - { - auto num_rows = block.rows(); - ColumnWithTypeAndName col; - col.type = std::make_shared(); - col.name = "__grouping_set"; - col.column = ColumnUInt64::create(num_rows, group); - block.insert(grouping_position, std::move(col)); - } -} +// void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) +// { +// auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); +// for (auto & block : block_list) +// { +// auto num_rows = block.rows(); +// ColumnWithTypeAndName col; +// col.type = std::make_shared(); +// col.name = "__grouping_set"; +// col.column = ColumnUInt64::create(num_rows, group); +// block.insert(grouping_position, std::move(col)); +// } +// } void MergingAggregatedTransform::consume(Chunk chunk) { @@ -170,6 +245,25 @@ void MergingAggregatedTransform::consume(Chunk chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); } +// static void debugBlock(const Block & block) +// { +// std::cerr << block.dumpStructure() << std::endl; +// size_t rows = block.rows(); +// for (size_t row = 0; row < rows; ++row) +// { +// for (size_t col = 0; col < block.columns(); ++col) +// { +// const auto & c = block.getByPosition(col); +// if (c.column->isNumeric()) +// std::cerr << c.column->getUInt(row) << ' '; +// else +// std::cerr << c.column->getDataAt(row).toString() << ' '; + +// } +// std::cerr << std::endl; +// } +// } + Chunk MergingAggregatedTransform::generate() { if (!generate_started) @@ -180,15 +274,31 @@ Chunk MergingAggregatedTransform::generate() /// Exception safety. Make iterator valid in case any method below throws. next_block = blocks.begin(); - for (auto & [group, group_blocks] : grouping_sets) + for (auto & grouping_set : grouping_sets) { - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + auto & params = grouping_set.params; + auto & bucket_to_blocks = grouping_set.bucket_to_blocks; AggregatedDataVariants data_variants; - params->aggregator.mergeBlocks(std::move(group_blocks), data_variants, max_threads, is_cancelled); + + // std::cerr << "== Group " << group << std::endl; + // for (const auto & [buk, lst] : bucket_to_blocks) + // { + // std::cerr << ".. buk " << buk << std::endl; + // for (const auto & b : lst) + // debugBlock(b); + // } + + /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); - if (has_grouping_sets) - appendGroupingColumn(group, merged_blocks); + if (grouping_set.creating_missing_keys_actions) + for (auto & block : merged_blocks) + grouping_set.creating_missing_keys_actions->execute(block); + + // std::cerr << "== Merged " << group << std::endl; + // for (const auto & b : merged_blocks) + // debugBlock(b); blocks.splice(blocks.end(), std::move(merged_blocks)); } diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index 1d801f7a94d..3a043ad74b8 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -6,13 +6,24 @@ namespace DB { +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; + /** A pre-aggregate stream of blocks in which each block is already aggregated. * Aggregate functions in blocks should not be finalized so that their states can be merged. */ class MergingAggregatedTransform : public IAccumulatingTransform { public: - MergingAggregatedTransform(Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_); + MergingAggregatedTransform( + Block header_, + Aggregator::Params params_, + bool final_, + GroupingSetsParamsList grouping_sets_params, + size_t max_threads_); + + ~MergingAggregatedTransform() override; + String getName() const override { return "MergingAggregatedTransform"; } static Block appendGroupingIfNeeded(const Block & in_header, Block out_header); @@ -22,13 +33,19 @@ protected: Chunk generate() override; private: - AggregatingTransformParamsPtr params; LoggerPtr log = getLogger("MergingAggregatedTransform"); size_t max_threads; - using GroupingSets = std::unordered_map; + struct GroupingSet + { + Aggregator::BucketToBlocks bucket_to_blocks; + ExpressionActionsPtr reordering_key_columns_actions; + ExpressionActionsPtr creating_missing_keys_actions; + AggregatingTransformParamsPtr params; + }; + + using GroupingSets = std::vector; GroupingSets grouping_sets; - const bool has_grouping_sets; UInt64 total_input_rows = 0; UInt64 total_input_blocks = 0; @@ -40,7 +57,6 @@ private: bool generate_started = false; void addBlock(Block block); - void appendGroupingColumn(UInt64 group, BlocksList & block_list); }; } diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference index 4589dc7d7a5..31cbf2ad670 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference @@ -11,57 +11,215 @@ 0 6 4 1 10 4 2 14 4 -['.'] -['.','.'] -['.','.','.'] -['.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -['.'] -['.'] -['.','.'] -['.','.'] -['.','.','.'] -['.','.','.'] -['.','.','.','.'] -['.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -1 -2 -3 -4 -5 -6 -7 -8 -9 -1 -1 -2 -2 -3 -3 -4 -4 -5 -5 -6 -6 -7 -7 -8 -8 -9 -9 +-- { echo On } + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 ['.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 ['.'] +1 ['.'] +2 ['.','.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +2 3 +2 3 +2 3 +2 4 +2 4 +2 4 +2 5 +2 5 +2 5 +2 6 +2 6 +2 6 +2 7 +2 7 +2 7 +2 8 +2 8 +2 8 +2 9 +2 9 +2 9 +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 ['.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 ['.'] +2 ['.'] +2 ['.','.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 1 +2 2 +2 2 +2 2 +2 3 +2 3 +2 3 +2 4 +2 4 +2 4 +2 5 +2 5 +2 5 +2 6 +2 6 +2 6 +2 7 +2 7 +2 7 +2 8 +2 8 +2 8 +2 9 +2 9 +2 9 diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql index 333dab79575..47d4446f348 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql @@ -44,7 +44,22 @@ ORDER BY sum_value ASC, count_value ASC; -SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +set prefer_localhost_replica = 1; + +-- { echo On } + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; From 2b495e22cdeea4c769c73b7e21f448be2c5ffcae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 15:38:52 +0000 Subject: [PATCH 382/409] Fixing a test. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index cf383cfcf9d..dd97364f879 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -111,7 +111,10 @@ void MergingAggregatedTransform::addBlock(Block block) { if (grouping_sets.size() == 1) { - grouping_sets[0].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + auto bucket = block.info.bucket_num; + if (grouping_sets[0].reordering_key_columns_actions) + grouping_sets[0].reordering_key_columns_actions->execute(block); + grouping_sets[0].bucket_to_blocks[bucket].emplace_back(std::move(block)); return; } From ac91471042ebac5fc5467aef9efe806124293f1a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Aug 2024 16:06:12 +0000 Subject: [PATCH 383/409] add comment --- src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 6caf69e3a2c..6f1c3937880 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -232,6 +232,8 @@ std::tuple getTypedNodesForOptimizati const auto & storage_snapshot = table_node->getStorageSnapshot(); auto column = first_argument_column_node->getColumn(); + /// If view source is set we cannot optimize because it doesn't support moving functions to subcolumns. + /// The storage is replaced to the view source but it happens only after building a query tree and applying passes. auto view_source = context->getViewSource(); if (view_source && view_source->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted()) return {}; From a7584bbb80c68917702a31492a5024faf7c2aaf4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 16:07:31 +0000 Subject: [PATCH 384/409] Remove comments. --- .../Transforms/MergingAggregatedTransform.cpp | 45 ------------------- 1 file changed, 45 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index dd97364f879..edd544fb6af 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -191,20 +191,6 @@ void MergingAggregatedTransform::addBlock(Block block) } } -// void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) -// { -// auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); -// for (auto & block : block_list) -// { -// auto num_rows = block.rows(); -// ColumnWithTypeAndName col; -// col.type = std::make_shared(); -// col.name = "__grouping_set"; -// col.column = ColumnUInt64::create(num_rows, group); -// block.insert(grouping_position, std::move(col)); -// } -// } - void MergingAggregatedTransform::consume(Chunk chunk) { if (!consume_started) @@ -248,25 +234,6 @@ void MergingAggregatedTransform::consume(Chunk chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); } -// static void debugBlock(const Block & block) -// { -// std::cerr << block.dumpStructure() << std::endl; -// size_t rows = block.rows(); -// for (size_t row = 0; row < rows; ++row) -// { -// for (size_t col = 0; col < block.columns(); ++col) -// { -// const auto & c = block.getByPosition(col); -// if (c.column->isNumeric()) -// std::cerr << c.column->getUInt(row) << ' '; -// else -// std::cerr << c.column->getDataAt(row).toString() << ' '; - -// } -// std::cerr << std::endl; -// } -// } - Chunk MergingAggregatedTransform::generate() { if (!generate_started) @@ -283,14 +250,6 @@ Chunk MergingAggregatedTransform::generate() auto & bucket_to_blocks = grouping_set.bucket_to_blocks; AggregatedDataVariants data_variants; - // std::cerr << "== Group " << group << std::endl; - // for (const auto & [buk, lst] : bucket_to_blocks) - // { - // std::cerr << ".. buk " << buk << std::endl; - // for (const auto & b : lst) - // debugBlock(b); - // } - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); @@ -299,10 +258,6 @@ Chunk MergingAggregatedTransform::generate() for (auto & block : merged_blocks) grouping_set.creating_missing_keys_actions->execute(block); - // std::cerr << "== Merged " << group << std::endl; - // for (const auto & b : merged_blocks) - // debugBlock(b); - blocks.splice(blocks.end(), std::move(merged_blocks)); } From a1517cb9d6598c6ae7cfef5d574702966ea244a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 16:56:03 +0000 Subject: [PATCH 385/409] Refactor a bit and add a comment. --- .../Transforms/MergingAggregatedTransform.cpp | 64 +++++++++++-------- 1 file changed, 37 insertions(+), 27 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index edd544fb6af..9b76acb8081 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -25,6 +25,42 @@ Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header return out_header; } +/// We should keep the order for GROUPING SET keys. +/// Initiator creates a separate Aggregator for every group, so should we do here. +/// Otherwise, two-level aggregation will split the data into different buckets, +/// and the result may have duplicating rows. +static ActionsDAG makeReorderingActions(const Block & in_header, const GroupingSetsParams & params) +{ + ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); + auto & outputs = reordering.getOutputs(); + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(in_header.columns() + params.used_keys.size() - params.used_keys.size()); + + std::unordered_map index; + for (size_t pos = 0; pos < outputs.size(); ++pos) + index.emplace(outputs[pos]->result_name, pos); + + for (const auto & used_name : params.used_keys) + { + auto & idx = index[used_name]; + new_outputs.push_back(outputs[idx]); + } + + for (const auto & used_name : params.used_keys) + index[used_name] = outputs.size(); + for (const auto & missing_name : params.missing_keys) + index[missing_name] = outputs.size(); + + for (const auto * output : outputs) + { + if (index[output->result_name] != outputs.size()) + new_outputs.push_back(output); + } + + outputs.swap(new_outputs); + return reordering; +} + MergingAggregatedTransform::~MergingAggregatedTransform() = default; MergingAggregatedTransform::MergingAggregatedTransform( @@ -52,33 +88,7 @@ MergingAggregatedTransform::MergingAggregatedTransform( { size_t group = grouping_sets.size(); - ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); - auto & outputs = reordering.getOutputs(); - ActionsDAG::NodeRawConstPtrs new_outputs; - new_outputs.reserve(in_header.columns() + grouping_set_params.used_keys.size() - grouping_set_params.used_keys.size()); - - std::unordered_map index; - for (size_t pos = 0; pos < outputs.size(); ++pos) - index.emplace(outputs[pos]->result_name, pos); - - for (const auto & used_name : grouping_set_params.used_keys) - { - auto & idx = index[used_name]; - new_outputs.push_back(outputs[idx]); - } - - for (const auto & used_name : grouping_set_params.used_keys) - index[used_name] = outputs.size(); - for (const auto & missing_name : grouping_set_params.missing_keys) - index[missing_name] = outputs.size(); - - for (const auto * output : outputs) - { - if (index[output->result_name] != outputs.size()) - new_outputs.push_back(output); - } - - outputs.swap(new_outputs); + auto reordering = makeReorderingActions(in_header, grouping_set_params); Aggregator::Params set_params(grouping_set_params.used_keys, params.aggregates, From 26e1338c4eaa47cc9d6e0aadb473a184d425f780 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 19:05:10 +0200 Subject: [PATCH 386/409] Update runner --- tests/integration/runner | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index cdd75438b87..83c28cff7c7 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -153,7 +153,7 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( """ - iptables is in 'legacy' mode. This is not supported. + iptables on your host machine is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. Or, use --ignore-iptables-legacy-check. @@ -170,7 +170,7 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( """ - ip6tables is in 'legacy' mode. This is not supported. + ip6tables on your host machine is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. Or, use --ignore-iptables-legacy-check. From ea1289a2b36e80c8662b2ab59b6c13cfbcb40526 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 22:42:14 +0000 Subject: [PATCH 387/409] Actually fix false "Killed by signal (output files)" in stress_tests.lib --- tests/docker_scripts/stress_tests.lib | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index 4f3e6eeb2f4..cc4c290afef 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -272,8 +272,10 @@ function check_logs_for_critical_errors() # Remove file no_such_key_errors.txt if it's empty [ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt - # Crash - rg -Fa "###################""#####################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ + # Crash. This must have fewer '#'s than the command below, otherwise the command below will match + # the echo of this command (if set -x is enabled, and this script's stdout is sent + # to /test_output/run.log). + rg -Fa "#######################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ && echo -e "Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \ || echo -e "Not crashed$OK" >> /test_output/test_results.tsv @@ -285,7 +287,7 @@ function check_logs_for_critical_errors() # Remove file fatal_messages.txt if it's empty [ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt - rg -Faz "####################""####################" /test_output/* > /dev/null \ + rg -Faz "########################################" /test_output/* > /dev/null \ && echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv function get_gdb_log_context() From ea32862187ade24499f4b441d133952d09dd49aa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 23:10:48 +0000 Subject: [PATCH 388/409] no-tsan in 02735_parquet_encoder --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 9320d0e57c3..26d0fefed1b 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel +-- Tags: no-fasttest, no-parallel, no-tsan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 61599108c7b91eaf558c0faaeac33c0928932a6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Aug 2024 01:37:38 +0200 Subject: [PATCH 389/409] Make 01114_database_atomic timings consistent --- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 412f5336224..4f635280666 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -51,8 +51,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='${DATABASE_2}'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" RANDOM_COMMENT="$RANDOM" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition [Using 5 threads in parallel]), result: 110, 5995 +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do @@ -87,7 +87,7 @@ SELECT count() FROM ${DATABASE_1}.mt " # result: 5 RANDOM_TUPLE="${RANDOM}_tuple" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do it=$((it+1)) From 1f10ba2901ef62f8578e75b9aa982ee94e358141 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2024 09:16:08 +0200 Subject: [PATCH 390/409] Update CHANGELOG.md Remove the server setting just in case, because its applicability was not clarified, and its name is not proper. --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 93dcfa18999..854a95820ab 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -30,7 +30,6 @@ * Support more variants of JOIN strictness (`LEFT/RIGHT SEMI/ANTI/ANY JOIN`) with inequality conditions which involve columns from both left and right table. e.g. `t1.y < t2.y` (see the setting `allow_experimental_join_condition`). [#64281](https://github.com/ClickHouse/ClickHouse/pull/64281) ([lgbo](https://github.com/lgbo-ustc)). * Intrpret Hive-style partitioning for different engines (`File`, `URL`, `S3`, `AzureBlobStorage`, `HDFS`). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). [#65997](https://github.com/ClickHouse/ClickHouse/pull/65997) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Add function `printf` for Spark compatiability (but you can use the existing `format` function). [#66257](https://github.com/ClickHouse/ClickHouse/pull/66257) ([李扬](https://github.com/taiyang-li)). -* Added a new server setting, `disable_insertion_and_mutation`. If it is enabled, the server will deny all insertions and mutations. This includes asynchronous INSERTs. This setting can be used to create read-only replicas. [#66519](https://github.com/ClickHouse/ClickHouse/pull/66519) ([Xu Jia](https://github.com/XuJia0210)). * Add options `restore_replace_external_engines_to_null` and `restore_replace_external_table_functions_to_null` to replace external engines and table_engines to `Null` engine that can be useful for testing. It should work for RESTORE and explicit table creation. [#66536](https://github.com/ClickHouse/ClickHouse/pull/66536) ([Ilya Yatsishin](https://github.com/qoega)). * Added support for reading `MULTILINESTRING` geometry in `WKT` format using function `readWKTLineString`. [#67647](https://github.com/ClickHouse/ClickHouse/pull/67647) ([Jacob Reckhard](https://github.com/jacobrec)). * Add a new table function `fuzzQuery`. This function allows the modification of a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;`. [#67655](https://github.com/ClickHouse/ClickHouse/pull/67655) ([pufit](https://github.com/pufit)). From 8a923d2b21e9becf39656da161e8778411d4dccd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 28 Aug 2024 09:58:50 +0200 Subject: [PATCH 391/409] Update 02735_parquet_encoder.sql MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 26d0fefed1b..8e0f7095a42 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-tsan +-- Tags: no-fasttest, no-parallel, no-tsan, no-msan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 0fe8f94b1561ac06370eb9500ac54044065b7df9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Aug 2024 12:50:25 +0200 Subject: [PATCH 392/409] Fix missing check --- tests/queries/0_stateless/01114_database_atomic.reference | 3 +++ tests/queries/0_stateless/01114_database_atomic.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 33080a7a77b..572cedb64c5 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -2,6 +2,9 @@ CREATE DATABASE default_1\nENGINE = Atomic CREATE DATABASE default_2\nENGINE = Atomic CREATE DATABASE default_3\nENGINE = Ordinary +default_1 Atomic store 00001114-1000-4000-8000-000000000001 1 +default_2 Atomic store 00001114-1000-4000-8000-000000000002 1 +default_3 Ordinary default_3 default_3 1 110 100 CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 4f635280666..fb4672ef906 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -29,7 +29,7 @@ $CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], splitByChar('/', metadata_path)[-2] as uuid_path, ((splitByChar('/', metadata_path)[-3] as metadata) = substr(uuid_path, 1, 3)) OR metadata='metadata' - FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" + FROM system.databases WHERE name LIKE '${CLICKHOUSE_DATABASE}_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE ${DATABASE_1}.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); From a842994af121631a055186e7d628650bbe3a3521 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 28 Aug 2024 15:21:49 +0200 Subject: [PATCH 393/409] fix false leak detect in libfiu --- contrib/libfiu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libfiu b/contrib/libfiu index b85edbde4cf..a1290d8cd3d 160000 --- a/contrib/libfiu +++ b/contrib/libfiu @@ -1 +1 @@ -Subproject commit b85edbde4cf974b1b40d27828a56f0505f4e2ee5 +Subproject commit a1290d8cd3d7b4541d6c976e0a54f572ac03f2a3 From 3314042278d938e7faff7029e7b810f64744c7bb Mon Sep 17 00:00:00 2001 From: Rodrigo Garcia Date: Wed, 28 Aug 2024 16:14:31 +0200 Subject: [PATCH 394/409] Link to CASE changed after explicit anchor removed --- docs/en/sql-reference/functions/conditional-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index 564186fd8db..8b8ee524c7e 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -49,7 +49,7 @@ Result: ## multiIf -Allows to write the [CASE](../../sql-reference/operators/index.md#operator_case) operator more compactly in the query. +Allows to write the [CASE](../../sql-reference/operators/index.md#conditional-expression) operator more compactly in the query. **Syntax** @@ -264,4 +264,4 @@ SELECT clamp(1, 2, 3) result, toTypeName(result) type; ┌─result─┬─type────┐ │ 2 │ Float64 │ └────────┴─────────┘ -``` \ No newline at end of file +``` From 42039331ada8b7b626b279b003acc5f2b972f78e Mon Sep 17 00:00:00 2001 From: okunev Date: Wed, 28 Aug 2024 17:47:43 +0200 Subject: [PATCH 395/409] Support for the Spanish language in the embedded dictionaries. --- src/Dictionaries/Embedded/RegionsNames.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/Embedded/RegionsNames.h b/src/Dictionaries/Embedded/RegionsNames.h index 0e4c1fe8b88..73b432fb30d 100644 --- a/src/Dictionaries/Embedded/RegionsNames.h +++ b/src/Dictionaries/Embedded/RegionsNames.h @@ -35,9 +35,10 @@ class RegionsNames M(et, ru, 11) \ M(pt, en, 12) \ M(he, en, 13) \ - M(vi, en, 14) + M(vi, en, 14) \ + M(es, en, 15) - static constexpr size_t total_languages = 15; + static constexpr size_t total_languages = 16; public: enum class Language : size_t From ca13b13d811b1999f8e75bf0b1596d37ed92510a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 17:54:42 +0200 Subject: [PATCH 396/409] Add CPU arch to short fault info --- src/Common/SignalHandlers.cpp | 4 ++-- src/Common/config_version.cpp.in | 2 ++ src/Common/config_version.h | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 6ac6cbcae29..fc82c50c073 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -371,8 +371,8 @@ try /// in case of double fault. LOG_FATAL(log, "########## Short fault info ############"); - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}, architecture: {}) (from thread {}) Received signal {}", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, SYSTEM_PROCESSOR, thread_num, sig); std::string signal_description = "Unknown signal"; diff --git a/src/Common/config_version.cpp.in b/src/Common/config_version.cpp.in index eb9ceb800b9..cc9a7021c6c 100644 --- a/src/Common/config_version.cpp.in +++ b/src/Common/config_version.cpp.in @@ -15,3 +15,5 @@ const char * VERSION_DESCRIBE = "@VERSION_DESCRIBE@"; const unsigned VERSION_INTEGER = @VERSION_INTEGER@; const char * VERSION_GITHASH = "@VERSION_GITHASH@"; + +const char * SYSTEM_PROCESSOR = "@CMAKE_SYSTEM_PROCESSOR@"; diff --git a/src/Common/config_version.h b/src/Common/config_version.h index e3ec12e2b34..4e1ed9388a7 100644 --- a/src/Common/config_version.h +++ b/src/Common/config_version.h @@ -16,6 +16,7 @@ extern const char * VERSION_OFFICIAL; extern const char * VERSION_FULL; extern const char * VERSION_DESCRIBE; extern const unsigned VERSION_INTEGER; +extern const char * SYSTEM_PROCESSOR; /// These fields are frequently changing and we don't want to have them in the header file to allow caching. extern const char * VERSION_GITHASH; From b6966f6c88efc7515c8eca0c6fdf5d947e7bfbe7 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 18:26:26 +0200 Subject: [PATCH 397/409] Use Poco --- src/Common/SignalHandlers.cpp | 3 ++- src/Common/config_version.cpp.in | 2 -- src/Common/config_version.h | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index fc82c50c073..554a56f22e8 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #pragma clang diagnostic ignored "-Wreserved-identifier" @@ -372,7 +373,7 @@ try LOG_FATAL(log, "########## Short fault info ############"); LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}, architecture: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, SYSTEM_PROCESSOR, + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, Poco::Environment::osArchitecture(), thread_num, sig); std::string signal_description = "Unknown signal"; diff --git a/src/Common/config_version.cpp.in b/src/Common/config_version.cpp.in index cc9a7021c6c..eb9ceb800b9 100644 --- a/src/Common/config_version.cpp.in +++ b/src/Common/config_version.cpp.in @@ -15,5 +15,3 @@ const char * VERSION_DESCRIBE = "@VERSION_DESCRIBE@"; const unsigned VERSION_INTEGER = @VERSION_INTEGER@; const char * VERSION_GITHASH = "@VERSION_GITHASH@"; - -const char * SYSTEM_PROCESSOR = "@CMAKE_SYSTEM_PROCESSOR@"; diff --git a/src/Common/config_version.h b/src/Common/config_version.h index 4e1ed9388a7..e3ec12e2b34 100644 --- a/src/Common/config_version.h +++ b/src/Common/config_version.h @@ -16,7 +16,6 @@ extern const char * VERSION_OFFICIAL; extern const char * VERSION_FULL; extern const char * VERSION_DESCRIBE; extern const unsigned VERSION_INTEGER; -extern const char * SYSTEM_PROCESSOR; /// These fields are frequently changing and we don't want to have them in the header file to allow caching. extern const char * VERSION_GITHASH; From c96eeb45f0aa0277fd56a8d50c4d3ea67a2070cf Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Wed, 28 Aug 2024 18:47:52 +0200 Subject: [PATCH 398/409] Update README.md Adding more meetups --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index ba212852ea8..d12f1a6ff37 100644 --- a/README.md +++ b/README.md @@ -56,6 +56,14 @@ Other upcoming meetups * [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 * [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 * [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 +* [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22 +* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 +* [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - October 29 +* [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 +* [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 +* [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 + + ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From d6127d5f4966b7fad22073e26ee466d654b7529b Mon Sep 17 00:00:00 2001 From: Konstantin Smirnov <46676677+konnectr@users.noreply.github.com> Date: Wed, 28 Aug 2024 22:30:12 +0500 Subject: [PATCH 399/409] add ON CLUSTER --- docs/ru/sql-reference/statements/system.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 3e7d67d90ff..d17e5acd693 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -280,7 +280,7 @@ SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge Ждет когда таблица семейства `ReplicatedMergeTree` будет синхронизирована с другими репликами в кластере, но не более `receive_timeout` секунд: ``` sql -SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL] +SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL] ``` После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все загруженные команды. Поддерживаются следующие модификаторы: From d390999912919c2ea261fe62619d4fcaae0b0521 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 28 Aug 2024 15:26:06 -0700 Subject: [PATCH 400/409] Update tests/queries/0_stateless/02735_parquet_encoder.sql --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 8e0f7095a42..c7e12a970fc 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-tsan, no-msan +-- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From cd9be01c65767ff88131c110b5a2daf0a663034e Mon Sep 17 00:00:00 2001 From: Aleksa Cukovic Date: Thu, 29 Aug 2024 14:40:38 +0200 Subject: [PATCH 401/409] Fix row policy documentation grammar --- .../statements/create/row-policy.md | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index cd7718793bd..8be766710fd 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -8,7 +8,7 @@ title: "CREATE ROW POLICY" Creates a [row policy](../../../guides/sre/user-management/index.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. :::tip -Row policies makes sense only for users with readonly access. If user can modify table or copy partitions between tables, it defeats the restrictions of row policies. +Row policies make sense only for users with readonly access. If a user can modify a table or copy partitions between tables, it defeats the restrictions of row policies. ::: Syntax: @@ -24,40 +24,40 @@ CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluste ## USING Clause -Allows to specify a condition to filter rows. An user will see a row if the condition is calculated to non-zero for the row. +Allows specifying a condition to filter rows. A user will see a row if the condition is calculated to non-zero for the row. ## TO Clause -In the section `TO` you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`. +In the `TO` section you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`. -Keyword `ALL` means all the ClickHouse users including current user. Keyword `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` +Keyword `ALL` means all the ClickHouse users, including current user. Keyword `ALL EXCEPT` allows excluding some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` :::note -If there are no row policies defined for a table then any user can `SELECT` all the row from the table. Defining one or more row policies for the table makes the access to the table depending on the row policies no matter if those row policies are defined for the current user or not. For example, the following policy +If there are no row policies defined for a table, then any user can `SELECT` all the rows from the table. Defining one or more row policies for the table makes access to the table dependent on the row policies, no matter if those row policies are defined for the current user or not. For example, the following policy: `CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter` -forbids the users `mira` and `peter` to see the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all. +forbids the users `mira` and `peter` from seeing the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all. -If that's not desirable it can't be fixed by adding one more row policy, like the following: +If that's not desirable, it can be fixed by adding one more row policy, like the following: `CREATE ROW POLICY pol2 ON mydb.table1 USING 1 TO ALL EXCEPT mira, peter` ::: ## AS Clause -It's allowed to have more than one policy enabled on the same table for the same user at the one time. So we need a way to combine the conditions from multiple policies. +It's allowed to have more than one policy enabled on the same table for the same user at one time. So we need a way to combine the conditions from multiple policies. -By default policies are combined using the boolean `OR` operator. For example, the following policies +By default, policies are combined using the boolean `OR` operator. For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 TO peter, antonio ``` -enables the user `peter` to see rows with either `b=1` or `c=2`. +enable the user `peter` to see rows with either `b=1` or `c=2`. -The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default policies are permissive, which means they are combined using the boolean `OR` operator. +The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default, policies are permissive, which means they are combined using the boolean `OR` operator. A policy can be defined as restrictive as an alternative. Restrictive policies are combined using the boolean `AND` operator. @@ -68,25 +68,25 @@ row_is_visible = (one or more of the permissive policies' conditions are non-zer (all of the restrictive policies's conditions are non-zero) ``` -For example, the following policies +For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio ``` -enables the user `peter` to see rows only if both `b=1` AND `c=2`. +enable the user `peter` to see rows only if both `b=1` AND `c=2`. Database policies are combined with table policies. -For example, the following policies +For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.* USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio ``` -enables the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although +enable the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although any other table in mydb would have only `b=1` policy applied for the user. From 93143b61169e751b256acc1ef83bc71470c56704 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 29 Aug 2024 19:11:41 +0200 Subject: [PATCH 402/409] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index d12f1a6ff37..652f5e0751d 100644 --- a/README.md +++ b/README.md @@ -62,6 +62,7 @@ Other upcoming meetups * [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 * [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 +* [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 From a474a5bc7c956a6a44b89b454dfe957b890466b6 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 29 Aug 2024 17:15:39 +0000 Subject: [PATCH 403/409] asan timed out too (because flaky check has timeout of ~36 seconds instead of 600) --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index c7e12a970fc..ab5047cb915 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan +-- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan, no-asan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 1772a877ad7a5518443b6851956ed787206f42ae Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 29 Aug 2024 20:10:34 +0200 Subject: [PATCH 404/409] fix logical error in async inserts --- src/Interpreters/AsynchronousInsertQueue.cpp | 20 +++++++++++++++---- .../02481_async_insert_dedup.python | 9 +++++++-- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..c9137f39426 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1004,8 +1004,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( size_t num_rows = executor.execute(*buffer); total_rows += num_rows; - chunk_info->offsets.push_back(total_rows); - chunk_info->tokens.push_back(entry->async_dedup_token); + /// for some reason, client can pass zero rows and bytes to server. + /// We don't update offsets in this case, because we assume every insert has some rows during dedup + /// but we have nothing to deduplicate for this insert. + if (num_rows > 0) + { + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + } add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes, data->timeout_ms); @@ -1056,8 +1062,14 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); total_rows += block->rows(); - chunk_info->offsets.push_back(total_rows); - chunk_info->tokens.push_back(entry->async_dedup_token); + /// for some reason, client can pass zero rows and bytes to server. + /// We don't update offsets in this case, because we assume every insert has some rows during dedup, + /// but we have nothing to deduplicate for this insert. + if (block->rows()) + { + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + } const auto & query_for_logging = get_query_by_format(entry->format); add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes(), data->timeout_ms); diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 24512836290..292d93833bf 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -48,9 +48,11 @@ def generate_data(q, total_number, use_token): partitions = ["2022-11-11 10:10:10", "2022-12-12 10:10:10"] last_number = 0 while True: - dup_simulate = random.randint(0, 3) + # 0 to simulate duplication + # 1 to simulate empty + simulate_flag = random.randint(0, 4) # insert old data randomly. 25% of them are dup. - if dup_simulate == 0: + if simulate_flag == 0: last_idx = len(old_data) - 1 if last_idx < 0: continue @@ -58,6 +60,9 @@ def generate_data(q, total_number, use_token): if idx < 0: idx = 0 q.put(old_data[idx]) + if simulate_flag == 1: + empty_insert_stmt = "insert into t_async_insert_dedup values format JSONEachRow" + q.put((empty_insert_stmt, token)) else: # insert new data. chunk_size = random.randint(1, max_chunk_size) From 1a28621b9f7030e23410d9c887ae4ec4418c2258 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 29 Aug 2024 21:59:09 +0200 Subject: [PATCH 405/409] fix style --- tests/queries/0_stateless/02481_async_insert_dedup.python | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 292d93833bf..b70fc574a00 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -61,7 +61,9 @@ def generate_data(q, total_number, use_token): idx = 0 q.put(old_data[idx]) if simulate_flag == 1: - empty_insert_stmt = "insert into t_async_insert_dedup values format JSONEachRow" + empty_insert_stmt = ( + "insert into t_async_insert_dedup values format JSONEachRow" + ) q.put((empty_insert_stmt, token)) else: # insert new data. From 429905afa1ec0fb6f934920c73e6927edb1f5a14 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 30 Aug 2024 00:44:03 +0200 Subject: [PATCH 406/409] fix token --- tests/queries/0_stateless/02481_async_insert_dedup.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index b70fc574a00..c8b5abc11b0 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -64,7 +64,7 @@ def generate_data(q, total_number, use_token): empty_insert_stmt = ( "insert into t_async_insert_dedup values format JSONEachRow" ) - q.put((empty_insert_stmt, token)) + q.put((empty_insert_stmt, "")) else: # insert new data. chunk_size = random.randint(1, max_chunk_size) From 4ac4806b14917ff608ca53bea8c043165e96e58d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 30 Aug 2024 08:07:21 +0200 Subject: [PATCH 407/409] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 0be6256aa50..5021cd390d8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2729,6 +2729,7 @@ timeZoneOffset timezones tinylog tmp +toBool toColumnTypeName toDate toDateOrDefault From 6143f99f7ad3a4dcb2ae6cd9646dadc7ff5cf23d Mon Sep 17 00:00:00 2001 From: sunny19930321 Date: Fri, 30 Aug 2024 15:02:09 +0800 Subject: [PATCH 408/409] fix: change alter query to ternary expression --- src/Interpreters/InterpreterDeleteQuery.cpp | 12 +++--------- src/Parsers/ASTDeleteQuery.h | 2 +- .../02352_lightweight_delete_in_partition.sql | 4 ++-- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 69aae3ae909..c0d840b8c5a 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -78,15 +78,9 @@ BlockIO InterpreterDeleteQuery::execute() String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " UPDATE `_row_exists` = 0 WHERE " + serializeAST(*delete_query.predicate); - if (delete_query.partition) - { - alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " UPDATE `_row_exists` = 0 IN PARTITION " + serializeAST(*delete_query.partition) + " WHERE " - + serializeAST(*delete_query.predicate); - } + + " UPDATE `_row_exists` = 0" + + (delete_query.partition ? " IN PARTITION " + serializeAST(*delete_query.partition) : "") + + " WHERE " + serializeAST(*delete_query.predicate); ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h index 6adc9b6a1c8..fcbeafac294 100644 --- a/src/Parsers/ASTDeleteQuery.h +++ b/src/Parsers/ASTDeleteQuery.h @@ -19,7 +19,7 @@ public: return removeOnCluster(clone(), params.default_database); } - /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE, DELETE FROM queries. + /** Used in DELETE FROM queries. * The value or ID of the partition is stored here. */ ASTPtr partition; diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql index 08cf5816f74..04371d273fd 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql @@ -5,10 +5,10 @@ CREATE TABLE t_merge_tree(time Date, id String , name String) ENGINE = MergeTree CREATE TABLE t_replicated_merge_tree(time Date, id String, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','1') PARTITION BY time ORDER BY id; INSERT INTO t_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); -INSERT INTO t_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); +INSERT INTO t_merge_tree select '2024-08-02', '1', toString(number) FROM numbers(100); INSERT INTO t_replicated_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); -INSERT INTO t_replicated_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); +INSERT INTO t_replicated_merge_tree select '2024-08-02', '1', toString(number) FROM numbers(100); SELECT COUNT() FROM t_merge_tree; SELECT COUNT() FROM t_replicated_merge_tree; From b0ecacc0c3082867286e7061ac831ec779079844 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 30 Aug 2024 12:53:42 +0200 Subject: [PATCH 409/409] remove no-parallel --- tests/queries/0_stateless/02481_async_insert_dedup.sh | 2 +- tests/queries/0_stateless/02481_async_insert_dedup_token.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.sh b/tests/queries/0_stateless/02481_async_insert_dedup.sh index 0fe06e6ab58..2a646a6bccd 100755 --- a/tests/queries/0_stateless/02481_async_insert_dedup.sh +++ b/tests/queries/0_stateless/02481_async_insert_dedup.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh index bb2d07066a5..81e315ce7dc 100755 --- a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh