From fb666e137c86339b532f9becc956eb4ef60971d3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 Oct 2023 23:25:20 +0000 Subject: [PATCH 001/116] remove old code of projection analysis --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 147 ++------ src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Interpreters/SelectQueryOptions.h | 20 - src/Planner/Utils.cpp | 3 +- .../Optimizations/optimizeReadInOrder.cpp | 5 +- .../QueryPlan/ReadFromMergeTree.cpp | 37 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 5 +- src/Storages/IStorage.cpp | 5 +- src/Storages/IStorage.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 5 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 348 +----------------- .../MergeTree/MergeTreeDataSelectExecutor.h | 1 - src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 +- .../ReadFinalForExternalReplicaStorage.cpp | 2 +- src/Storages/SelectQueryInfo.h | 42 --- src/Storages/StorageBuffer.cpp | 8 +- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 4 - src/Storages/StorageMerge.cpp | 6 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageProxy.h | 2 - src/Storages/StorageReplicatedMergeTree.cpp | 16 +- src/Storages/StorageReplicatedMergeTree.h | 2 - 27 files changed, 79 insertions(+), 602 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 5b26084e440..b581d152369 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1040,7 +1040,7 @@ static std::unique_ptr buildJoinedPlan( join_element.table_expression, context, original_right_column_names, - query_options.copy().setWithAllColumns().ignoreProjections(false).ignoreAlias(false)); + query_options.copy().setWithAllColumns().ignoreAlias(false)); auto joined_plan = std::make_unique(); interpreter->buildQueryPlan(*joined_plan); { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5e7ece5912f..cb8653907d3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -386,8 +386,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!prepared_sets) prepared_sets = std::make_shared(); - query_info.ignore_projections = options.ignore_projections; - query_info.is_projection_query = options.is_projection_query; query_info.is_internal = options.is_internal; initSettings(); @@ -413,7 +411,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( } query_info.query = query_ptr->clone(); - query_info.original_query = query_ptr->clone(); if (settings.count_distinct_optimization) { @@ -854,9 +851,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( analysis_result.required_columns = required_columns; } - if (query_info.projection) - storage_snapshot->addProjection(query_info.projection->desc); - /// Blocks used in expression analysis contains size 1 const columns for constant folding and /// null non-const columns to avoid useless memory allocations. However, a valid block sample /// requires all columns to be of size 0, thus we need to sanitize the block here. @@ -868,10 +862,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) executeImpl(query_plan, std::move(input_pipe)); /// We must guarantee that result structure is the same as in getSampleBlock() - /// - /// But if it's a projection query, plan header does not match result_header. - /// TODO: add special stage for InterpreterSelectQuery? - if (!options.is_projection_query && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) + if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) { auto convert_actions_dag = ActionsDAG::makeConvertingActions( query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), @@ -1370,12 +1361,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

desc->type == ProjectionDescription::Type::Aggregate) - { - query_info.projection->aggregate_overflow_row = aggregate_overflow_row; - query_info.projection->aggregate_final = aggregate_final; - } - if (options.only_analyze) { auto read_nothing = std::make_unique(source_header); @@ -1444,11 +1429,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

{}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } - if (query_info.projection && query_info.projection->input_order_info && query_info.input_order_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "InputOrderInfo is set for projection and for query"); InputOrderInfoPtr input_order_info_for_order; if (!expressions.need_aggregate) - input_order_info_for_order = query_info.projection ? query_info.projection->input_order_info : query_info.input_order_info; + input_order_info_for_order = query_info.input_order_info; if (options.to_stage > QueryProcessingStage::FetchColumns) { @@ -1505,7 +1488,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), @@ -1679,7 +1662,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(source_header)); - PrewhereInfoPtr prewhere_info_ptr = query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info; - if (prewhere_info_ptr) + if (query_info.prewhere_info) { - auto & prewhere_info = *prewhere_info_ptr; + auto & prewhere_info = *query_info.prewhere_info; if (prewhere_info.row_level_filter) { @@ -1978,50 +1959,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); query_plan.addStep(std::move(read_from_pipe)); - - if (query_info.projection) - { - if (query_info.projection->before_where) - { - auto where_step = std::make_unique( - query_plan.getCurrentDataStream(), - query_info.projection->before_where, - query_info.projection->where_column_name, - query_info.projection->remove_where_filter); - - where_step->setStepDescription("WHERE"); - query_plan.addStep(std::move(where_step)); - } - - if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) - { - if (query_info.projection->before_aggregation) - { - auto expression_before_aggregation - = std::make_unique(query_plan.getCurrentDataStream(), query_info.projection->before_aggregation); - expression_before_aggregation->setStepDescription("Before GROUP BY"); - query_plan.addStep(std::move(expression_before_aggregation)); - } - - // Let's just choose the safe option since we don't know the value of `to_stage` here. - const bool should_produce_results_in_order_of_bucket_number = true; - - // It is used to determine if we should use memory bound merging strategy. Maybe it makes sense for projections, but so far this case is just left untouched. - SortDescription group_by_sort_description; - - executeMergeAggregatedImpl( - query_plan, - query_info.projection->aggregate_overflow_row, - query_info.projection->aggregate_final, - false, - false, - context_->getSettingsRef(), - query_info.projection->aggregation_keys, - query_info.projection->aggregate_descriptions, - should_produce_results_in_order_of_bucket_number, - std::move(group_by_sort_description)); - } - } } RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const @@ -2428,56 +2365,28 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Create optimizer with prepared actions. /// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge. - if ((optimize_read_in_order || optimize_aggregation_in_order) - && (!query_info.projection || query_info.projection->complete)) + if (optimize_read_in_order || optimize_aggregation_in_order) { if (optimize_read_in_order) { - if (query_info.projection) - { - query_info.projection->order_optimizer = std::make_shared( - // TODO Do we need a projection variant for this field? - query, - analysis_result.order_by_elements_actions, - getSortDescription(query, context), - query_info.syntax_analyzer_result); - } - else - { - query_info.order_optimizer = std::make_shared( - query, - analysis_result.order_by_elements_actions, - getSortDescription(query, context), - query_info.syntax_analyzer_result); - } + query_info.order_optimizer = std::make_shared( + query, + analysis_result.order_by_elements_actions, + getSortDescription(query, context), + query_info.syntax_analyzer_result); } - else if (optimize_aggregation_in_order) + else { - if (query_info.projection) - { - query_info.projection->order_optimizer = std::make_shared( - query, - query_info.projection->group_by_elements_actions, - query_info.projection->group_by_elements_order_descr, - query_info.syntax_analyzer_result); - } - else - { - query_info.order_optimizer = std::make_shared( - query, - analysis_result.group_by_elements_actions, - getSortDescriptionFromGroupBy(query), - query_info.syntax_analyzer_result); - } + query_info.order_optimizer = std::make_shared( + query, + analysis_result.group_by_elements_actions, + getSortDescriptionFromGroupBy(query), + query_info.syntax_analyzer_result); } /// If we don't have filtration, we can pushdown limit to reading stage for optimizations. UInt64 limit = (query.hasFiltration() || query.groupBy()) ? 0 : getLimitForSorting(query, context); - if (query_info.projection) - query_info.projection->input_order_info - = query_info.projection->order_optimizer->getInputOrder(query_info.projection->desc->metadata, context, limit); - else - query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, limit); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, limit); } query_info.storage_limits = std::make_shared(storage_limits); @@ -2493,7 +2402,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc backQuoteIfNeed(local_storage_id.getDatabaseName()), local_storage_id.getFullTableName(), required_columns, - query_info.projection ? query_info.projection->desc->name : "", + /*projection_name=*/ "", view_name); } @@ -2501,7 +2410,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (!query_plan.isInitialized()) { auto header = storage_snapshot->getSampleBlockForColumns(required_columns); - addEmptySourceToQueryPlan(query_plan, header, query_info, context); + addEmptySourceToQueryPlan(query_plan, header, query_info); } } else @@ -2609,13 +2518,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac expression_before_aggregation->setStepDescription("Before GROUP BY"); query_plan.addStep(std::move(expression_before_aggregation)); - if (options.is_projection_query) - return; - AggregateDescriptions aggregates = query_analyzer->aggregates(); - const Settings & settings = context->getSettingsRef(); - const auto & keys = query_analyzer->aggregationKeys().getNames(); auto aggregator_params = getAggregatorParams( @@ -2679,13 +2583,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final, bool has_grouping_sets) { - /// If aggregate projection was chosen for table, avoid adding MergeAggregated. - /// It is already added by storage (because of performance issues). - /// TODO: We should probably add another one processing stage for storage? - /// WithMergeableStateAfterAggregation is not ok because, e.g., it skips sorting after aggregation. - if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) - return; - const Settings & settings = context->getSettingsRef(); /// Used to determine if we should use memory bound merging strategy. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 41f43f4c4b4..9ea3972977f 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -117,7 +117,7 @@ public: bool hasAggregation() const { return query_analyzer->hasAggregation(); } static void addEmptySourceToQueryPlan( - QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_); + QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info); Names getRequiredColumns() { return required_columns; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2db4fce81f0..3bcd1e24797 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -178,7 +178,7 @@ bool isStorageTouchedByMutations( if (context->getSettingsRef().allow_experimental_analyzer) { auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); - InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits().ignoreProjections()); + InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits()); io = interpreter.execute(); } else @@ -188,7 +188,7 @@ bool isStorageTouchedByMutations( /// For some reason it may copy context and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. interpreter_select_query.emplace( - select_query, context, storage_from_part, metadata_snapshot, SelectQueryOptions().ignoreLimits().ignoreProjections()); + select_query, context, storage_from_part, metadata_snapshot, SelectQueryOptions().ignoreLimits()); io = interpreter_select_query->execute(); } @@ -367,7 +367,7 @@ MutationsInterpreter::MutationsInterpreter( , available_columns(std::move(available_columns_)) , context(Context::createCopy(context_)) , settings(std::move(settings_)) - , select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits().ignoreProjections()) + , select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits()) { prepare(!settings.can_execute); } diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index c91329c869c..1e08aec3813 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -33,14 +33,6 @@ struct SelectQueryOptions bool remove_duplicates = false; bool ignore_quota = false; bool ignore_limits = false; - /// This flag is needed to analyze query ignoring table projections. - /// It is needed because we build another one InterpreterSelectQuery while analyzing projections. - /// It helps to avoid infinite recursion. - bool ignore_projections = false; - /// This flag is also used for projection analysis. - /// It is needed because lazy normal projections require special planning in FetchColumns stage, such as adding WHERE transform. - /// It is also used to avoid adding aggregating step when aggregate projection is chosen. - bool is_projection_query = false; /// This flag is needed for projection description. /// Otherwise, keys for GROUP BY may be removed as constants. bool ignore_ast_optimizations = false; @@ -119,18 +111,6 @@ struct SelectQueryOptions return *this; } - SelectQueryOptions & ignoreProjections(bool value = true) - { - ignore_projections = value; - return *this; - } - - SelectQueryOptions & projectionQuery(bool value = true) - { - is_projection_query = value; - return *this; - } - SelectQueryOptions & ignoreAlias(bool value = true) { ignore_alias = value; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 733db0f00bc..ae94092de4b 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -419,8 +419,7 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context) { SelectQueryInfo select_query_info; - select_query_info.original_query = queryNodeToSelectQuery(query_tree); - select_query_info.query = select_query_info.original_query; + select_query_info.query = queryNodeToSelectQuery(query_tree); select_query_info.query_tree = query_tree; select_query_info.planner_context = planner_context; return select_query_info; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 655cb1fdb80..c970589ea33 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1073,10 +1073,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, /// If we don't have filtration, we can pushdown limit to reading stage for optimizations. UInt64 limit = (select_query->hasFiltration() || select_query->groupBy()) ? 0 : InterpreterSelectQuery::getLimitForSorting(*select_query, context); - auto order_info = order_optimizer->getInputOrder( - query_info.projection ? query_info.projection->desc->metadata : read_from_merge_tree->getStorageMetadata(), - context, - limit); + auto order_info = order_optimizer->getInputOrder(read_from_merge_tree->getStorageMetadata(), context, limit); if (order_info) { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 10f51563e9b..f743f9171f8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -121,12 +121,6 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( }; } -static const PrewhereInfoPtr & getPrewhereInfoFromQueryInfo(const SelectQueryInfo & query_info) -{ - return query_info.projection ? query_info.projection->prewhere_info - : query_info.prewhere_info; -} - static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) { for (const auto & part : parts) @@ -253,7 +247,7 @@ ReadFromMergeTree::ReadFromMergeTree( bool enable_parallel_reading) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(real_column_names_), - getPrewhereInfoFromQueryInfo(query_info_), + query_info_.prewhere_info, data_.getPartitionValueType(), virt_column_names_)}) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) @@ -263,7 +257,7 @@ ReadFromMergeTree::ReadFromMergeTree( , virt_column_names(std::move(virt_column_names_)) , data(data_) , query_info(query_info_) - , prewhere_info(getPrewhereInfoFromQueryInfo(query_info)) + , prewhere_info(query_info_.prewhere_info) , actions_settings(ExpressionActionsSettings::fromContext(context_)) , storage_snapshot(std::move(storage_snapshot_)) , metadata_for_reading(storage_snapshot->getMetadataForQuery()) @@ -318,7 +312,7 @@ ReadFromMergeTree::ReadFromMergeTree( *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo(), + query_info.input_order_info, prewhere_info); } @@ -1618,10 +1612,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( result.total_marks_pk = total_marks_pk; result.selected_rows = sum_rows; - const auto & input_order_info = query_info.getInputOrderInfo(); - if (input_order_info) - result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder - : ReadType::InReverseOrder; + if (query_info.input_order_info) + result.read_type = (query_info.input_order_info->direction > 0) + ? ReadType::InOrder + : ReadType::InReverseOrder; return std::make_shared(MergeTreeDataSelectAnalysisResult{.result = std::move(result)}); } @@ -1637,12 +1631,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, if (direction != 1 && query_info.isFinal()) return false; - auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); - if (query_info.projection) - query_info.projection->input_order_info = order_info; - else - query_info.input_order_info = order_info; - + query_info.input_order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); reader_settings.read_in_order = true; /// In case or read-in-order, don't create too many reading streams. @@ -1664,7 +1653,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, } if (!sort_description.empty()) { - const size_t used_prefix_of_sorting_key_size = order_info->used_prefix_of_sorting_key_size; + const size_t used_prefix_of_sorting_key_size = query_info.input_order_info->used_prefix_of_sorting_key_size; if (sort_description.size() > used_prefix_of_sorting_key_size) sort_description.resize(used_prefix_of_sorting_key_size); output_stream->sort_description = std::move(sort_description); @@ -1694,7 +1683,7 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo(), + query_info.input_order_info, prewhere_info); } @@ -1789,8 +1778,6 @@ Pipe ReadFromMergeTree::spreadMarkRanges( RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) { const bool final = isQueryWithFinal(); - const auto & input_order_info = query_info.getInputOrderInfo(); - Names column_names_to_read = result.column_names_to_read; NameSet names(column_names_to_read.begin(), column_names_to_read.end()); @@ -1831,10 +1818,10 @@ Pipe ReadFromMergeTree::spreadMarkRanges( return spreadMarkRangesAmongStreamsFinal(std::move(parts_with_ranges), num_streams, result.column_names_to_read, column_names_to_read, result_projection); } - else if (input_order_info) + else if (query_info.input_order_info) { return spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), num_streams, column_names_to_read, result_projection, input_order_info); + std::move(parts_with_ranges), num_streams, column_names_to_read, result_projection, query_info.input_order_info); } else { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index ab8a37e0323..ee8f8895bed 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -229,9 +229,8 @@ private: int getSortDirection() const { - const InputOrderInfoPtr & order_info = query_info.getInputOrderInfo(); - if (order_info) - return order_info->direction; + if (query_info.input_order_info) + return query_info.input_order_info->direction; return 1; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ae7659e074f..a435eb498d9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -149,7 +149,7 @@ void IStorage::read( if (parallelize_output && parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams) pipe.resize(num_streams); - readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName()); + readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, getName()); } void IStorage::readFromPipe( @@ -158,13 +158,12 @@ void IStorage::readFromPipe( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, std::string storage_name) { if (pipe.empty()) { auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); } else { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fcf7675d15d..eee7bf7aa1d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -681,7 +681,6 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, std::string storage_name); private: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 889dcfa537f..a4e88ebe936 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6596,11 +6596,6 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( return QueryProcessingStage::Enum::WithMergeableState; } - if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) - { - query_info.projection = std::nullopt; - } - return QueryProcessingStage::Enum::FetchColumns; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7e028f8c65c..5c9226ec370 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -124,22 +124,6 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTSampleRatio::Ra return std::min(RelativeSize(1), RelativeSize(absolute_sample_size) / RelativeSize(approx_total_rows)); } -static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) -{ - SortDescription order_descr; - order_descr.reserve(query.groupBy()->children.size()); - - for (const auto & elem : query.groupBy()->children) - { - /// Note, here aliases should not be used, since there will be no such column in a block. - String name = elem->getColumnNameWithoutAlias(); - order_descr.emplace_back(name, 1, 1); - } - - return order_descr; -} - - QueryPlanPtr MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, @@ -147,336 +131,32 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( ContextPtr context, const UInt64 max_block_size, const size_t num_streams, - QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read, bool enable_parallel_reading) const { if (query_info.merge_tree_empty_result) return std::make_unique(); - const auto & settings = context->getSettingsRef(); - - const auto & metadata_for_reading = storage_snapshot->getMetadataForQuery(); - const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; const auto & alter_conversions = snapshot_data.alter_conversions; - if (!query_info.projection) - { - auto step = readFromParts( - query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts, - query_info.merge_tree_select_result_ptr ? std::vector{} : alter_conversions, - column_names_to_return, - storage_snapshot, - query_info, - context, - max_block_size, - num_streams, - max_block_numbers_to_read, - query_info.merge_tree_select_result_ptr, - enable_parallel_reading); + auto step = readFromParts( + parts, + alter_conversions, + column_names_to_return, + storage_snapshot, + query_info, + context, + max_block_size, + num_streams, + max_block_numbers_to_read, + /*merge_tree_select_result_ptr=*/ nullptr, + enable_parallel_reading); - auto plan = std::make_unique(); - if (step) - plan->addStep(std::move(step)); - return plan; - } - - LOG_DEBUG( - log, - "Choose {} {} projection {}", - query_info.projection->complete ? "complete" : "incomplete", - query_info.projection->desc->type, - query_info.projection->desc->name); - - const ASTSelectQuery & select_query = query_info.query->as(); - QueryPlanResourceHolder resources; - - auto projection_plan = std::make_unique(); - if (query_info.projection->desc->is_minmax_count_projection) - { - Pipe pipe(std::make_shared(query_info.minmax_count_projection_block)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - projection_plan->addStep(std::move(read_from_pipe)); - } - else if (query_info.projection->merge_tree_projection_select_result_ptr) - { - LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); - projection_plan->addStep(readFromParts( - /*parts=*/ {}, - /*alter_conversions=*/ {}, - query_info.projection->required_columns, - storage_snapshot, - query_info, - context, - max_block_size, - num_streams, - max_block_numbers_to_read, - query_info.projection->merge_tree_projection_select_result_ptr, - enable_parallel_reading)); - } - - if (projection_plan->isInitialized()) - { - if (query_info.projection->before_where) - { - auto where_step = std::make_unique( - projection_plan->getCurrentDataStream(), - query_info.projection->before_where, - query_info.projection->where_column_name, - query_info.projection->remove_where_filter); - - where_step->setStepDescription("WHERE"); - projection_plan->addStep(std::move(where_step)); - } - - if (query_info.projection->before_aggregation) - { - auto expression_before_aggregation - = std::make_unique(projection_plan->getCurrentDataStream(), query_info.projection->before_aggregation); - expression_before_aggregation->setStepDescription("Before GROUP BY"); - projection_plan->addStep(std::move(expression_before_aggregation)); - } - - /// NOTE: input_order_info (for projection and not) is set only if projection is complete - if (query_info.has_order_by && !query_info.need_aggregate && query_info.projection->input_order_info) - { - chassert(query_info.projection->complete); - - SortDescription output_order_descr = InterpreterSelectQuery::getSortDescription(select_query, context); - UInt64 limit = InterpreterSelectQuery::getLimitForSorting(select_query, context); - - auto sorting_step = std::make_unique( - projection_plan->getCurrentDataStream(), - query_info.projection->input_order_info->sort_description_for_merging, - output_order_descr, - settings.max_block_size, - limit); - - sorting_step->setStepDescription("ORDER BY for projections"); - projection_plan->addStep(std::move(sorting_step)); - } - } - - auto ordinary_query_plan = std::make_unique(); - if (query_info.projection->merge_tree_normal_select_result_ptr) - { - auto storage_from_base_parts_of_projection - = std::make_shared(data, query_info.projection->merge_tree_normal_select_result_ptr); - auto interpreter = InterpreterSelectQuery( - query_info.query, - context, - storage_from_base_parts_of_projection, - nullptr, - SelectQueryOptions{processed_stage}.projectionQuery()); - - interpreter.buildQueryPlan(*ordinary_query_plan); - - const auto & expressions = interpreter.getAnalysisResult(); - if (processed_stage == QueryProcessingStage::Enum::FetchColumns && expressions.before_where) - { - auto where_step = std::make_unique( - ordinary_query_plan->getCurrentDataStream(), - expressions.before_where, - expressions.where_column_name, - expressions.remove_where_filter); - where_step->setStepDescription("WHERE"); - ordinary_query_plan->addStep(std::move(where_step)); - } - } - - Pipe projection_pipe; - Pipe ordinary_pipe; - if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) - { - auto make_aggregator_params = [&](bool projection) - { - const auto & keys = query_info.projection->aggregation_keys.getNames(); - - AggregateDescriptions aggregates = query_info.projection->aggregate_descriptions; - - /// This part is hacky. - /// We want AggregatingTransform to work with aggregate states instead of normal columns. - /// It is almost the same, just instead of adding new data to aggregation state we merge it with existing. - /// - /// It is needed because data in projection: - /// * is not merged completely (we may have states with the same key in different parts) - /// * is not split into buckets (so if we just use MergingAggregated, it will use single thread) - const bool only_merge = projection; - - Aggregator::Params params( - keys, - aggregates, - query_info.projection->aggregate_overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - settings.group_by_two_level_threshold, - settings.group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set, - context->getTempDataOnDisk(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression, - settings.max_block_size, - settings.enable_software_prefetch_in_aggregation, - only_merge, - settings.optimize_group_by_constant_keys); - - return std::make_pair(params, only_merge); - }; - - if (ordinary_query_plan->isInitialized() && projection_plan->isInitialized()) - { - auto projection_builder = projection_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources); - - auto ordinary_builder = ordinary_query_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources); - - /// Here we create shared ManyAggregatedData for both projection and ordinary data. - /// For ordinary data, AggregatedData is filled in a usual way. - /// For projection data, AggregatedData is filled by merging aggregation states. - /// When all AggregatedData is filled, we merge aggregation states together in a usual way. - /// Pipeline will look like: - /// ReadFromProjection -> Aggregating (only merge states) -> - /// ReadFromProjection -> Aggregating (only merge states) -> - /// ... -> Resize -> ConvertingAggregatedToChunks - /// ReadFromOrdinaryPart -> Aggregating (usual) -> (added by last Aggregating) - /// ReadFromOrdinaryPart -> Aggregating (usual) -> - /// ... - auto many_data = std::make_shared(projection_pipe.numOutputPorts() + ordinary_pipe.numOutputPorts()); - size_t counter = 0; - - AggregatorListPtr aggregator_list_ptr = std::make_shared(); - - /// TODO apply optimize_aggregation_in_order here too (like below) - auto build_aggregate_pipe = [&](Pipe & pipe, bool projection) - { - auto [params, only_merge] = make_aggregator_params(projection); - - AggregatingTransformParamsPtr transform_params = std::make_shared( - pipe.getHeader(), std::move(params), aggregator_list_ptr, query_info.projection->aggregate_final); - - pipe.resize(pipe.numOutputPorts(), true, true); - - auto merge_threads = num_streams; - auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, transform_params, many_data, counter++, merge_threads, temporary_data_merge_threads); - }); - }; - - if (!projection_pipe.empty()) - build_aggregate_pipe(projection_pipe, true); - if (!ordinary_pipe.empty()) - build_aggregate_pipe(ordinary_pipe, false); - } - else - { - auto add_aggregating_step = [&](QueryPlanPtr & query_plan, bool projection) - { - auto [params, only_merge] = make_aggregator_params(projection); - - auto merge_threads = num_streams; - auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads); - - InputOrderInfoPtr group_by_info = query_info.projection->input_order_info; - SortDescription sort_description_for_merging; - SortDescription group_by_sort_description; - if (group_by_info && settings.optimize_aggregation_in_order) - { - group_by_sort_description = getSortDescriptionFromGroupBy(select_query); - sort_description_for_merging = group_by_info->sort_description_for_merging; - } - else - group_by_info = nullptr; - - // We don't have information regarding the `to_stage` of the query processing, only about `from_stage` (which is passed through `processed_stage` argument). - // Thus we cannot assign false here since it may be a query over distributed table. - const bool should_produce_results_in_order_of_bucket_number = true; - - auto aggregating_step = std::make_unique( - query_plan->getCurrentDataStream(), - std::move(params), - /* grouping_sets_params_= */ GroupingSetsParamsList{}, - query_info.projection->aggregate_final, - settings.max_block_size, - settings.aggregation_in_order_max_block_bytes, - merge_threads, - temporary_data_merge_threads, - /* storage_has_evenly_distributed_read_= */ false, - /* group_by_use_nulls */ false, - std::move(sort_description_for_merging), - std::move(group_by_sort_description), - should_produce_results_in_order_of_bucket_number, - settings.enable_memory_bound_merging_of_aggregation_results, - !group_by_info && settings.force_aggregation_in_order); - query_plan->addStep(std::move(aggregating_step)); - }; - - if (projection_plan->isInitialized()) - { - add_aggregating_step(projection_plan, true); - - auto projection_builder = projection_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources); - } - if (ordinary_query_plan->isInitialized()) - { - add_aggregating_step(ordinary_query_plan, false); - - auto ordinary_builder = ordinary_query_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources); - } - } - } - else - { - if (projection_plan->isInitialized()) - { - auto projection_builder = projection_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources); - } - - if (ordinary_query_plan->isInitialized()) - { - auto ordinary_builder = ordinary_query_plan->buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources); - } - } - - Pipes pipes; - pipes.emplace_back(std::move(projection_pipe)); - pipes.emplace_back(std::move(ordinary_pipe)); - auto pipe = Pipe::unitePipes(std::move(pipes)); auto plan = std::make_unique(); - if (pipe.empty()) - return plan; - - pipe.resize(1); - auto step = std::make_unique( - std::move(pipe), - fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name), - query_info.storage_limits); - plan->addStep(std::move(step)); - plan->addInterpreterContext(query_info.projection->context); + if (step) + plan->addStep(std::move(step)); return plan; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index d5d8107db48..1e57fd71bd1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -34,7 +34,6 @@ public: ContextPtr context, UInt64 max_block_size, size_t num_streams, - QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read = nullptr, bool enable_parallel_reading = false) const; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index a3478069356..92f4abdf390 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -342,7 +342,7 @@ void StorageNATS::read( if (pipe.empty()) { auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); } else { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index ec552dd1032..5a11bdf5fc0 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -695,7 +695,7 @@ void StorageRabbitMQ::read( if (num_created_consumers == 0) { auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); return; } @@ -753,7 +753,7 @@ void StorageRabbitMQ::read( if (pipe.empty()) { auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); } else { diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 28053c84e20..e1d52eefc20 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -64,7 +64,7 @@ void readFinalFromNestedStorage( if (!query_plan.isInitialized()) { - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, nested_header, query_info, context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, nested_header, query_info); return; } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 6d52d45c6a9..69dbb64db38 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -9,7 +9,6 @@ #include #include #include -#include #include @@ -142,32 +141,6 @@ class IMergeTreeDataPart; using ManyExpressionActions = std::vector; -// The projection selected to execute current query -struct ProjectionCandidate -{ - ProjectionDescriptionRawPtr desc{}; - PrewhereInfoPtr prewhere_info; - ActionsDAGPtr before_where; - String where_column_name; - bool remove_where_filter = false; - ActionsDAGPtr before_aggregation; - Names required_columns; - NamesAndTypesList aggregation_keys; - AggregateDescriptions aggregate_descriptions; - bool aggregate_overflow_row = false; - bool aggregate_final = false; - bool complete = false; - ReadInOrderOptimizerPtr order_optimizer; - InputOrderInfoPtr input_order_info; - ManyExpressionActions group_by_elements_actions; - SortDescription group_by_elements_order_descr; - MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; - MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; - - /// Because projection analysis uses a separate interpreter. - ContextPtr context; -}; - /** Query along with some additional data, * that can be used during query processing * inside storage engines. @@ -180,7 +153,6 @@ struct SelectQueryInfo ASTPtr query; ASTPtr view_query; /// Optimized VIEW query - ASTPtr original_query; /// Unmodified query for projection analysis /// Query tree QueryTreeNodePtr query_tree; @@ -242,20 +214,11 @@ struct SelectQueryInfo ClusterPtr getCluster() const { return !optimized_cluster ? cluster : optimized_cluster; } - /// If not null, it means we choose a projection to execute current query. - std::optional projection; - bool ignore_projections = false; - bool is_projection_query = false; bool merge_tree_empty_result = false; bool settings_limit_offset_done = false; bool is_internal = false; - Block minmax_count_projection_block; - MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr; - bool parallel_replicas_disabled = false; - bool is_parameterized_view = false; - bool optimize_trivial_count = false; // If limit is not 0, that means it's a trivial limit query. @@ -264,11 +227,6 @@ struct SelectQueryInfo /// For IStorageSystemOneBlock std::vector columns_mask; - InputOrderInfoPtr getInputOrderInfo() const - { - return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr); - } - bool isFinal() const; }; } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e011565edc1..b27b28e5a53 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -210,8 +210,6 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage( { if (auto destination = getDestinationTable()) { - /// TODO: Find a way to support projections for StorageBuffer - query_info.ignore_projections = true; const auto & destination_metadata = destination->getInMemoryMetadataPtr(); return destination->getQueryProcessingStage(local_context, to_stage, destination->getStorageSnapshot(destination_metadata, local_context), query_info); } @@ -335,12 +333,12 @@ void StorageBuffer::read( pipes_from_buffers.emplace_back(std::make_shared(column_names, buf, storage_snapshot)); pipe_from_buffers = Pipe::unitePipes(std::move(pipes_from_buffers)); - if (query_info.getInputOrderInfo()) + if (query_info.input_order_info) { /// Each buffer has one block, and it not guaranteed that rows in each block are sorted by order keys pipe_from_buffers.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.getInputOrderInfo()->sort_description_for_merging, 0); + return std::make_shared(header, query_info.input_order_info->sort_description_for_merging, 0); }); } } @@ -358,7 +356,7 @@ void StorageBuffer::read( /// TODO: Find a way to support projections for StorageBuffer auto interpreter = InterpreterSelectQuery( query_info.query, local_context, std::move(pipe_from_buffers), - SelectQueryOptions(processed_stage).ignoreProjections()); + SelectQueryOptions(processed_stage)); interpreter.addStorageLimits(*query_info.storage_limits); interpreter.buildQueryPlan(buffers_plan); } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index df03301b5e8..ee3b63f8586 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -166,7 +166,7 @@ void StorageExecutable::read( } auto pipe = coordinator->createPipe(script_path, settings.script_arguments, std::move(inputs), std::move(sample_block), context, configuration); - IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName()); + IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, getName()); query_plan.addResources(std::move(resources)); } diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index d493fead993..beb93afc972 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -73,7 +73,7 @@ void StorageExternalDistributed::read( if (plans.empty()) { auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); } if (plans.size() == 1) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 7354dd56552..3c68219d6d4 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -134,10 +134,6 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( const StorageSnapshotPtr &, SelectQueryInfo & query_info) const { - /// TODO: Find a way to support projections for StorageMaterializedView. Why do we use different - /// metadata for materialized view and target table? If they are the same, we can get rid of all - /// converting and use it just like a normal view. - query_info.ignore_projections = true; const auto & target_metadata = getTargetTable()->getInMemoryMetadataPtr(); return getTargetTable()->getQueryProcessingStage(local_context, to_stage, getTargetTable()->getStorageSnapshot(target_metadata, local_context), query_info); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b6253fa6daf..ad1808d90a0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -272,8 +272,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( size_t selected_table_size = 0; - /// TODO: Find a way to support projections for StorageMerge - query_info.ignore_projections = true; for (const auto & iterator : database_table_iterators) { while (iterator->isValid()) @@ -711,7 +709,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree, modified_context, - SelectQueryOptions(processed_stage).ignoreProjections()); + SelectQueryOptions(processed_stage)); builder = std::make_unique(interpreter.buildQueryPipeline()); plan = std::move(interpreter.getPlanner()).extractQueryPlan(); } @@ -721,7 +719,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// TODO: Find a way to support projections for StorageMerge InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, - SelectQueryOptions(processed_stage).ignoreProjections()}; + SelectQueryOptions(processed_stage)}; builder = std::make_unique(interpreter.buildQueryPipeline(plan)); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 470e30b7947..0f0093b6da9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -254,7 +254,7 @@ void StorageMergeTree::read( if (auto plan = reader.read( column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, - processed_stage, nullptr, enable_parallel_reading)) + nullptr, enable_parallel_reading)) query_plan = std::move(*plan); } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index a4304faeaec..2fb5bbde79b 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -38,8 +38,6 @@ public: const StorageSnapshotPtr &, SelectQueryInfo & info) const override { - /// TODO: Find a way to support projections for StorageProxy - info.ignore_projections = true; const auto & nested_metadata = getNested()->getInMemoryMetadataPtr(); return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata, context), info); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 63f785a93e5..35d5925e465 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5150,12 +5150,12 @@ void StorageReplicatedMergeTree::read( /// 2. Do not read parts that have not yet been written to the quorum of the replicas. /// For this you have to synchronously go to ZooKeeper. if (settings.select_sequential_consistency) - return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); if (local_context->canUseParallelReplicasOnInitiator()) return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); - readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); } void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( @@ -5164,14 +5164,15 @@ void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) { auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); - auto plan = reader.read(column_names, storage_snapshot, query_info, local_context, - max_block_size, num_streams, processed_stage, std::move(max_added_blocks), - /* enable_parallel_reading= */false); + auto plan = reader.read( + column_names, storage_snapshot, query_info, local_context, + max_block_size, num_streams, std::move(max_added_blocks), + /* enable_parallel_reading=*/ false); + if (plan) query_plan = std::move(*plan); } @@ -5232,16 +5233,15 @@ void StorageReplicatedMergeTree::readLocalImpl( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, const size_t max_block_size, const size_t num_streams) { auto plan = reader.read( column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, - processed_stage, /* max_block_numbers_to_read= */ nullptr, /* enable_parallel_reading= */ local_context->canUseParallelReplicasOnFollower()); + if (plan) query_plan = std::move(*plan); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 794991d8e06..9e342c044bd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -564,7 +564,6 @@ private: const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams); @@ -574,7 +573,6 @@ private: const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams); From a01acf5d2a650fa7e7d4a2a7426fbdc29c5142c5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 13 Oct 2023 03:18:43 +0000 Subject: [PATCH 002/116] remove projection from StorageSnapshot --- .../Passes/ShardNumColumnToFunctionPass.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 6 +-- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizeUseAggregateProjection.cpp | 42 +++++++++---------- .../optimizeUseNormalProjection.cpp | 24 +++++------ .../QueryPlan/ReadFromMergeTree.cpp | 11 ++--- .../QueryPlan/ReadFromPreparedSource.cpp | 7 +--- .../QueryPlan/ReadFromPreparedSource.h | 3 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageSnapshot.cpp | 10 ++--- src/Storages/StorageSnapshot.h | 8 ---- 14 files changed, 50 insertions(+), 73 deletions(-) diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp index 52c30b7b35d..f3ff7fe06b1 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -46,7 +46,7 @@ public: return; const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - if (!storage->isVirtualColumn(column.name, storage_snapshot->getMetadataForQuery())) + if (!storage->isVirtualColumn(column.name, storage_snapshot->metadata)) return; auto function_node = std::make_shared("shardNum"); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 67187be962c..6fe5ec2d55a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1114,7 +1114,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select { for (const auto & name_type : storage_virtuals) { - if (name_type.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery())) + if (name_type.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->metadata)) { has_virtual_shard_num = true; break; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c95671da6be..d0b850d987d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -391,20 +391,20 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info /// We evaluate sampling for Merge lazily so we need to get all the columns if (storage_snapshot->storage.getName() == "Merge") { - const auto columns = storage_snapshot->getMetadataForQuery()->getColumns().getAll(); + const auto columns = storage_snapshot->metadata->getColumns().getAll(); for (const auto & column : columns) required_columns.insert(column.name); } else { - auto columns_required_for_sampling = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForSampling(); + auto columns_required_for_sampling = storage_snapshot->metadata->getColumnsRequiredForSampling(); required_columns.insert(columns_required_for_sampling.begin(), columns_required_for_sampling.end()); } } if (table_expression_modifiers->hasFinal()) { - auto columns_required_for_final = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForFinal(); + auto columns_required_for_final = storage_snapshot->metadata->getColumnsRequiredForFinal(); required_columns.insert(columns_required_for_final.begin(), columns_required_for_final.end()); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 3352567943a..d79cf65c366 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -137,7 +137,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) { - const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); + const auto & sampling_key = storage_snapshot->metadata->getSamplingKey(); const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes(); for (const auto & column : sampling_source_columns) required_columns_after_filter.push_back(ColumnWithTypeAndName(column.type, column.name)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 0599a0fa369..3e9d3e06493 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -612,6 +612,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & return false; } + Context::QualifiedProjectionName projection_name; QueryPlanStepPtr projection_reading; bool has_ordinary_parts; @@ -622,26 +623,21 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & // candidates.minmax_projection->block.dumpStructure()); Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); - projection_reading = std::make_unique( - std::move(pipe), - context, - query_info.is_internal - ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName - { - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = candidates.minmax_projection->candidate.projection->name, - }); + projection_reading = std::make_unique(std::move(pipe)); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) reading->resetParts(std::move(candidates.minmax_projection->normal_parts)); + + projection_name = Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = candidates.minmax_projection->candidate.projection->name, + }; } else { auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); - proj_snapshot->addProjection(best_candidate->projection); + auto proj_snapshot = std::make_shared(storage_snapshot->storage, best_candidate->projection->metadata); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -663,23 +659,23 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & { auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); Pipe pipe(std::make_shared(std::move(header))); - projection_reading = std::make_unique( - std::move(pipe), - context, - query_info.is_internal - ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName - { - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = best_candidate->projection->name, - }); + projection_reading = std::make_unique(std::move(pipe)); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; if (has_ordinary_parts) reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); + + projection_name = Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }; } + if (!query_info.is_internal && context->hasQueryContext()) + context->getQueryContext()->addQueryAccessInfo(projection_name); + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 727afcb1a99..c326ff43c9e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -164,9 +164,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); - proj_snapshot->addProjection(best_candidate->projection); + auto proj_snapshot = std::make_shared(storage_snapshot->storage, best_candidate->projection->metadata); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -187,16 +185,16 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!projection_reading) { Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); - projection_reading = std::make_unique( - std::move(pipe), - context, - query_info.is_internal - ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName - { - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = best_candidate->projection->name, - }); + projection_reading = std::make_unique(std::move(pipe)); + } + + if (!query_info.is_internal && context->hasQueryContext()) + { + context->getQueryContext()->addQueryAccessInfo(Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f743f9171f8..e7943fca112 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -260,7 +260,7 @@ ReadFromMergeTree::ReadFromMergeTree( , prewhere_info(query_info_.prewhere_info) , actions_settings(ExpressionActionsSettings::fromContext(context_)) , storage_snapshot(std::move(storage_snapshot_)) - , metadata_for_reading(storage_snapshot->getMetadataForQuery()) + , metadata_for_reading(storage_snapshot->metadata) , context(std::move(context_)) , block_size{ .max_block_size_rows = max_block_size_, @@ -310,7 +310,7 @@ ReadFromMergeTree::ReadFromMergeTree( updateSortDescriptionForOutputStream( *output_stream, - storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + storage_snapshot->metadata->getSortingKeyColumns(), getSortDirection(), query_info.input_order_info, prewhere_info); @@ -1681,7 +1681,7 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info updateSortDescriptionForOutputStream( *output_stream, - storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + storage_snapshot->metadata->getSortingKeyColumns(), getSortDirection(), query_info.input_order_info, prewhere_info); @@ -1884,11 +1884,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons partition_names.emplace_back( fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); } - context->getQueryContext()->addQueryAccessInfo(partition_names); - if (storage_snapshot->projection) - context->getQueryContext()->addQueryAccessInfo( - Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); + context->getQueryContext()->addQueryAccessInfo(partition_names); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index a24c4dbe4d0..7446203ec35 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -4,19 +4,14 @@ namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) - , context(std::move(context_)) - , qualified_projection_name(std::move(qualified_projection_name_)) { } void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - if (context && context->hasQueryContext()) - context->getQueryContext()->addQueryAccessInfo(qualified_projection_name); - for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 2606f501009..03831ef6207 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -11,8 +11,7 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource( - Pipe pipe_, ContextPtr context_ = nullptr, Context::QualifiedProjectionName qualified_projection_name_ = {}); + explicit ReadFromPreparedSource(Pipe pipe_); String getName() const override { return "ReadFromPreparedSource"; } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f5f0fa6f726..e0a0a3313fe 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -70,7 +70,7 @@ bool injectRequiredColumnsRecursively( /// Column doesn't have default value and don't exist in part /// don't need to add to required set. - auto metadata_snapshot = storage_snapshot->getMetadataForQuery(); + auto metadata_snapshot = storage_snapshot->metadata; const auto column_default = metadata_snapshot->getColumns().getDefault(column_name); if (!column_default) return false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a4e88ebe936..1579409e3a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6612,7 +6612,7 @@ bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis( auto result_ptr = reader.estimateNumMarksToRead( parts, query_info.prewhere_info, - storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), + storage_snapshot->metadata->getColumns().getAll().getNames(), storage_snapshot->metadata, storage_snapshot->metadata, query_info, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ad1808d90a0..c3019b49f1f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -436,7 +436,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu query_info.input_order_info = input_sorting_info; } - auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); + auto sample_block = merge_storage_snapshot->metadata->getSampleBlock(); std::vector> pipelines; QueryPlanResourceHolder resources; diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 0c19634f50c..3df4ad4a862 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -30,7 +30,7 @@ void StorageSnapshot::init() NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const { - auto all_columns = getMetadataForQuery()->getColumns().get(options); + auto all_columns = metadata->getColumns().get(options); if (options.with_extended_objects) extendObjectColumns(all_columns, object_columns, options.with_subcolumns); @@ -77,7 +77,7 @@ NamesAndTypesList StorageSnapshot::getColumnsByNames(const GetColumnsOptions & o std::optional StorageSnapshot::tryGetColumn(const GetColumnsOptions & options, const String & column_name) const { - const auto & columns = getMetadataForQuery()->getColumns(); + const auto & columns = metadata->getColumns(); auto column = columns.tryGetColumn(options, column_name); if (column && (!column->type->hasDynamicSubcolumns() || !options.with_extended_objects)) return column; @@ -119,7 +119,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons { Block res; - const auto & columns = getMetadataForQuery()->getColumns(); + const auto & columns = metadata->getColumns(); for (const auto & column_name : column_names) { auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); @@ -151,7 +151,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons ColumnsDescription StorageSnapshot::getDescriptionForColumns(const Names & column_names) const { ColumnsDescription res; - const auto & columns = getMetadataForQuery()->getColumns(); + const auto & columns = metadata->getColumns(); for (const auto & name : column_names) { auto column = columns.tryGetColumnOrSubcolumnDescription(GetColumnsOptions::All, name); @@ -188,7 +188,7 @@ namespace void StorageSnapshot::check(const Names & column_names) const { - const auto & columns = getMetadataForQuery()->getColumns(); + const auto & columns = metadata->getColumns(); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(); if (column_names.empty()) diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index a69f9b95955..a07479f9372 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -25,9 +25,6 @@ struct StorageSnapshot using DataPtr = std::unique_ptr; DataPtr data; - /// Projection that is used in query. - mutable const ProjectionDescription * projection = nullptr; - StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_) @@ -81,11 +78,6 @@ struct StorageSnapshot DataTypePtr getConcreteType(const String & column_name) const; - void addProjection(const ProjectionDescription * projection_) const { projection = projection_; } - - /// If we have a projection then we should use its metadata. - StorageMetadataPtr getMetadataForQuery() const { return projection ? projection->metadata : metadata; } - private: void init(); From 7e3587034d17dd75838110bb3236816161bcd3f7 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Thu, 12 Oct 2023 19:49:06 +0100 Subject: [PATCH 003/116] Consume leading zeroes when parsing a number in ConstantExpressionTemplate --- .../Formats/Impl/ConstantExpressionTemplate.cpp | 4 ++++ .../02896_leading_zeroes_no_octal.reference | 15 +++++++++++++++ .../0_stateless/02896_leading_zeroes_no_octal.sql | 8 ++++++++ 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference create mode 100644 tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index fe6fb42d0a0..c2b0ce3e486 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -597,6 +597,10 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType( if (negative || *istr.position() == '+') ++istr.position(); + /// Consume leading zeroes - we don't want any funny octal business + while (*istr.position() == '0') + ++istr.position(); + static constexpr size_t MAX_LENGTH_OF_NUMBER = 319; char buf[MAX_LENGTH_OF_NUMBER + 1]; size_t bytes_to_copy = std::min(istr.available(), MAX_LENGTH_OF_NUMBER); diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference new file mode 100644 index 00000000000..5982c315878 --- /dev/null +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference @@ -0,0 +1,15 @@ +-0.02 0 +0 0 +0.01 0 +0.02 1 +0.03 1 +0.04 -1 +1 1 +2 5 +3 8 +4 17 +5 21 +6 51 +7 123 +8 16 +9 43981 diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql new file mode 100644 index 00000000000..fbb931b4c88 --- /dev/null +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS t_leading_zeroes; +CREATE TABLE t_leading_zeroes(ref Float64, val INTEGER) ENGINE=MergeTree ORDER BY ref; + +INSERT INTO t_leading_zeroes VALUES (-0.02, 00000), (0, 0), (0.01, 00), (0.02, 01), (0.03, +01), (0.04, -01), (1, 0001), (2, 0005), (3, 0008), (4, 0017), (5, 0021), (6, 0051), (7, 00000123), (8, 0b10000), (9, 0x0abcd); + +SELECT * FROM t_leading_zeroes; + +DROP TABLE IF EXISTS t_leading_zeroes; From 6ad027ed7a5d6377f36d60f103159a9f78c0e301 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 05:32:36 +0300 Subject: [PATCH 004/116] Update ConstantExpressionTemplate.cpp --- src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index c2b0ce3e486..421d41f93c9 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -598,7 +598,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType( ++istr.position(); /// Consume leading zeroes - we don't want any funny octal business - while (*istr.position() == '0') + while (!istr.eof() && *istr.position() == '0') ++istr.position(); static constexpr size_t MAX_LENGTH_OF_NUMBER = 319; From 69896d94bead717ebc75bdf1907935e6057ae98c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 4 Dec 2023 21:40:19 +0000 Subject: [PATCH 005/116] fix tests --- src/Interpreters/Context.cpp | 18 +++++++++++------- src/Interpreters/Context.h | 9 +++++---- src/Interpreters/InterpreterSelectQuery.cpp | 5 +---- src/Planner/PlannerJoinTree.cpp | 4 +--- .../QueryPlan/ReadFromPreparedSource.cpp | 18 +++++++++++++++--- .../QueryPlan/ReadFromPreparedSource.h | 14 ++------------ .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 6 +++++- src/Storages/IStorage.cpp | 5 +++-- src/Storages/IStorage.h | 1 + src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/StorageExecutable.cpp | 2 +- 13 files changed, 48 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0a8a8f1f529..70d0e6e9b27 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1553,9 +1553,7 @@ bool Context::hasScalar(const String & name) const void Context::addQueryAccessInfo( const String & quoted_database_name, const String & full_quoted_table_name, - const Names & column_names, - const String & projection_name, - const String & view_name) + const Names & column_names) { if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); @@ -1563,12 +1561,9 @@ void Context::addQueryAccessInfo( std::lock_guard lock(query_access_info.mutex); query_access_info.databases.emplace(quoted_database_name); query_access_info.tables.emplace(full_quoted_table_name); + for (const auto & column_name : column_names) query_access_info.columns.emplace(full_quoted_table_name + "." + backQuoteIfNeed(column_name)); - if (!projection_name.empty()) - query_access_info.projections.emplace(full_quoted_table_name + "." + backQuoteIfNeed(projection_name)); - if (!view_name.empty()) - query_access_info.views.emplace(view_name); } void Context::addQueryAccessInfo(const Names & partition_names) @@ -1581,6 +1576,15 @@ void Context::addQueryAccessInfo(const Names & partition_names) query_access_info.partitions.emplace(partition_name); } +void Context::addViewAccessInfo(const String & view_name) +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); + + std::lock_guard lock(query_access_info.mutex); + query_access_info.views.emplace(view_name); +} + void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name) { if (!qualified_projection_name) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8c169dd664f..5ec555c5596 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -690,13 +690,14 @@ public: void addSpecialScalar(const String & name, const Block & block); const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; } + void addQueryAccessInfo( const String & quoted_database_name, const String & full_quoted_table_name, - const Names & column_names, - const String & projection_name = {}, - const String & view_name = {}); + const Names & column_names); + void addQueryAccessInfo(const Names & partition_names); + void addViewAccessInfo(const String & view_name); struct QualifiedProjectionName { @@ -704,8 +705,8 @@ public: String projection_name; explicit operator bool() const { return !projection_name.empty(); } }; - void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name); + void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name); /// Supported factories for records in query_log enum class QueryLogFactories diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 26ec8c2ce34..98fae8e362c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2493,14 +2493,11 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (context->hasQueryContext() && !options.is_internal) { - const String view_name{}; auto local_storage_id = storage->getStorageID(); context->getQueryContext()->addQueryAccessInfo( backQuoteIfNeed(local_storage_id.getDatabaseName()), local_storage_id.getFullTableName(), - required_columns, - /*projection_name=*/ "", - view_name); + required_columns); } /// Create step which reads from empty source if storage has no data. diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 99b4468c63c..c063d36e288 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -841,9 +841,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_context->getQueryContext()->addQueryAccessInfo( backQuoteIfNeed(local_storage_id.getDatabaseName()), local_storage_id.getFullTableName(), - columns_names, - {}, - {}); + columns_names); } } diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index d0a97f5c74b..38945f4945a 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -20,11 +20,23 @@ void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, pipeline.init(std::move(pipe)); } +ReadFromStorageStep::ReadFromStorageStep( + Pipe pipe_, + String storage_name, + ContextPtr context_, + const SelectQueryInfo & query_info_) + : ReadFromPreparedSource(std::move(pipe_)) + , context(std::move(context_)) + , query_info(query_info_) +{ + setStepDescription(storage_name); + + for (const auto & processor : pipe.getProcessors()) + processor->setStorageLimits(query_info.storage_limits); +} + void ReadFromStorageStep::applyFilters() { - if (!context) - return; - std::shared_ptr key_condition; if (!context->getSettingsRef().allow_experimental_analyzer) { diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 3b6ff2f0696..2eea48553b3 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -16,32 +16,22 @@ public: explicit ReadFromPreparedSource(Pipe pipe_); String getName() const override { return "ReadFromPreparedSource"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; protected: Pipe pipe; - ContextPtr context; - Context::QualifiedProjectionName qualified_projection_name; }; class ReadFromStorageStep : public ReadFromPreparedSource { public: - ReadFromStorageStep(Pipe pipe_, String storage_name, const SelectQueryInfo & query_info_) - : ReadFromPreparedSource(std::move(pipe_)), query_info(query_info_) - { - setStepDescription(storage_name); - - for (const auto & processor : pipe.getProcessors()) - processor->setStorageLimits(query_info.storage_limits); - } + ReadFromStorageStep(Pipe pipe_, String storage_name, ContextPtr context_, const SelectQueryInfo & query_info_); String getName() const override { return "ReadFromStorage"; } - void applyFilters() override; private: + ContextPtr context; SelectQueryInfo query_info; }; diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 41690c1b132..67b7e3f4a80 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -331,7 +331,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep( , storage{std::move(storage_)} , storage_snapshot{storage_snapshot_} , context{std::move(context_)} - , key_expression{KeyDescription::parse(column_names[0], storage_snapshot->getMetadataForQuery()->columns, context).expression} + , key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context).expression} , max_block_size{max_block_size_} , num_streams{num_streams_} , limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context)) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ea10b025e87..d808966bb6a 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -405,7 +405,11 @@ Chain buildPushingToViewsChain( if (!no_destination && context->hasQueryContext()) { context->getQueryContext()->addQueryAccessInfo( - backQuoteIfNeed(view_id.getDatabaseName()), views_data->views.back().runtime_stats->target_name, {}, "", view_id.getFullTableName()); + backQuoteIfNeed(view_id.getDatabaseName()), + views_data->views.back().runtime_stats->target_name, + /*column_names=*/ {}); + + context->getQueryContext()->addViewAccessInfo(view_id.getFullTableName()); } } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 16fc0f6c09b..ce8176c1fc1 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -150,7 +150,7 @@ void IStorage::read( if (parallelize_output && parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams) pipe.resize(num_streams); - readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, getName()); + readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName()); } void IStorage::readFromPipe( @@ -159,6 +159,7 @@ void IStorage::readFromPipe( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, + ContextPtr context, std::string storage_name) { if (pipe.empty()) @@ -168,7 +169,7 @@ void IStorage::readFromPipe( } else { - auto read_step = std::make_unique(std::move(pipe), storage_name, query_info); + auto read_step = std::make_unique(std::move(pipe), storage_name, context, query_info); query_plan.addStep(std::move(read_step)); } } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1e6fbc6a75d..2a705b801da 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -731,6 +731,7 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, + ContextPtr context, std::string storage_name); private: diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index cd4d6382bee..9cb1fbd8506 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -351,7 +351,7 @@ void StorageNATS::read( } else { - auto read_step = std::make_unique(std::move(pipe), getName(), query_info); + auto read_step = std::make_unique(std::move(pipe), getName(), local_context, query_info); query_plan.addStep(std::move(read_step)); query_plan.addInterpreterContext(modified_context); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 03c46f8699e..fce2d775b15 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -762,7 +762,7 @@ void StorageRabbitMQ::read( } else { - auto read_step = std::make_unique(std::move(pipe), getName(), query_info); + auto read_step = std::make_unique(std::move(pipe), getName(), local_context, query_info); query_plan.addStep(std::move(read_step)); query_plan.addInterpreterContext(modified_context); } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index c3d1f39a9f6..2acbf3f4610 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -170,7 +170,7 @@ void StorageExecutable::read( } auto pipe = coordinator->createPipe(script_path, settings.script_arguments, std::move(inputs), std::move(sample_block), context, configuration); - IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, getName()); + IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName()); query_plan.addResources(std::move(resources)); } From 0aeb9fba192ed9678b102cbbe2a7a9578d59a689 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Tue, 5 Dec 2023 21:55:43 +0000 Subject: [PATCH 006/116] Move consuming of leading zeros from ReadBuffer to char buffer --- .../Impl/ConstantExpressionTemplate.cpp | 19 ++++----- .../02896_leading_zeroes_no_octal.reference | 40 ++++++++++++------- .../02896_leading_zeroes_no_octal.sql | 20 ++++++++-- 3 files changed, 52 insertions(+), 27 deletions(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 1c009252183..fe11e3f6360 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -597,28 +597,29 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType( if (negative || *istr.position() == '+') ++istr.position(); - /// Consume leading zeroes - we don't want any funny octal business - while (!istr.eof() && *istr.position() == '0') - ++istr.position(); - static constexpr size_t MAX_LENGTH_OF_NUMBER = 319; char buf[MAX_LENGTH_OF_NUMBER + 1]; size_t bytes_to_copy = std::min(istr.available(), MAX_LENGTH_OF_NUMBER); memcpy(buf, istr.position(), bytes_to_copy); buf[bytes_to_copy] = 0; - char * pos_double = buf; + /// Consume leading zeroes - we don't want any funny octal business + auto* non_zero_buf = buf; + while (*non_zero_buf == '0') + ++non_zero_buf; + + char * pos_double = non_zero_buf; errno = 0; - Float64 float_value = std::strtod(buf, &pos_double); - if (pos_double == buf || errno == ERANGE || float_value < 0) + Float64 float_value = std::strtod(non_zero_buf, &pos_double); + if (pos_double == non_zero_buf || errno == ERANGE || float_value < 0) return false; if (negative) float_value = -float_value; - char * pos_integer = buf; + char * pos_integer = non_zero_buf; errno = 0; - UInt64 uint_value = std::strtoull(buf, &pos_integer, 0); + UInt64 uint_value = std::strtoull(non_zero_buf, &pos_integer, 0); if (pos_integer == pos_double && errno != ERANGE && (!negative || uint_value <= (1ULL << 63))) { istr.position() += pos_integer - buf; diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference index 5982c315878..69446796e09 100644 --- a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference @@ -1,15 +1,25 @@ --0.02 0 -0 0 -0.01 0 -0.02 1 -0.03 1 -0.04 -1 -1 1 -2 5 -3 8 -4 17 -5 21 -6 51 -7 123 -8 16 -9 43981 +Leading zeroes into INTEGER +1 1 00000 0 0 +1 2 0 0 0 +1 3 00 0 0 +1 4 01 1 1 +1 5 +01 1 1 +1 6 -01 -1 -1 +1 7 0001 1 1 +1 8 0005 5 5 +1 9 0008 8 8 +1 10 0017 17 17 +1 11 0021 21 21 +1 12 0051 51 51 +1 13 00000123 123 123 +1 14 0b10000 16 16 +1 15 0x0abcd 43981 43981 +1 16 0000.008 0 0 +1 17 1000.0008 1000 1000 +1 18 0008.0008 8 8 +Leading zeroes into Float32 +1 1 00000 0 0 +1 2 00009.00009 9.00009 9.00009 +1 3 00009e9 9000000000 9000000000 +1 4 00009e09 9000000000 9000000000 +1 5 00009e0009 9000000000 9000000000 diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql index fbb931b4c88..a204259f3ec 100644 --- a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql @@ -1,8 +1,22 @@ DROP TABLE IF EXISTS t_leading_zeroes; -CREATE TABLE t_leading_zeroes(ref Float64, val INTEGER) ENGINE=MergeTree ORDER BY ref; +CREATE TABLE t_leading_zeroes(id INTEGER, input String, val INTEGER, expected INTEGER) ENGINE=MergeTree ORDER BY id; -INSERT INTO t_leading_zeroes VALUES (-0.02, 00000), (0, 0), (0.01, 00), (0.02, 01), (0.03, +01), (0.04, -01), (1, 0001), (2, 0005), (3, 0008), (4, 0017), (5, 0021), (6, 0051), (7, 00000123), (8, 0b10000), (9, 0x0abcd); +INSERT INTO t_leading_zeroes VALUES (1, '00000', 00000, 0), (2, '0', 0, 0), (3, '00', 00, 0), (4, '01', 01, 1), (5, '+01', +01, 1); +INSERT INTO t_leading_zeroes VALUES (6, '-01', -01, -1), (7, '0001', 0001, 1), (8, '0005', 0005, 5), (9, '0008', 0008, 8); +INSERT INTO t_leading_zeroes VALUES (10, '0017', 0017, 17), (11, '0021', 0021, 21), (12, '0051', 0051, 51), (13, '00000123', 00000123, 123); +INSERT INTO t_leading_zeroes VALUES (14, '0b10000', 0b10000, 16), (15, '0x0abcd', 0x0abcd, 43981), (16, '0000.008', 0000.008, 0) +INSERT INTO t_leading_zeroes VALUES (17, '1000.0008', 1000.0008, 1000), (18, '0008.0008', 0008.0008, 8); -SELECT * FROM t_leading_zeroes; +SELECT 'Leading zeroes into INTEGER'; +SELECT t.val == t.expected AS ok, * FROM t_leading_zeroes t ORDER BY id; + +CREATE TABLE t_leading_zeroes_f(id INTEGER, input String, val Float32, expected Float32) ENGINE=MergeTree ORDER BY id; +INSERT INTO t_leading_zeroes_f VALUES (1, '00000', 00000, 0), (2, '00009.00009', 00009.00009, 9.00009), (3, '00009e9', 00009e9, 9e9), (4, '00009e09', 00009e09, 9e9), (5, '00009e0009', 00009e0009, 9e9); +-- Turns out this is not ok in master as well - will have a look and fix +-- (6, '00009e00009', 00009e00009, 9e9); + +SELECT 'Leading zeroes into Float32'; +SELECT t.val == t.expected AS ok, * FROM t_leading_zeroes_f t ORDER BY id; DROP TABLE IF EXISTS t_leading_zeroes; +DROP TABLE IF EXISTS t_leading_zeroes_f; \ No newline at end of file From f302ba0e31d03388a152cfe2764f5f7b39abff6e Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Dec 2023 17:47:18 -0500 Subject: [PATCH 007/116] Implement hardware-assisted(QAT) ZSTD compression --- .gitmodules | 6 + contrib/CMakeLists.txt | 22 +++ contrib/QAT-ZSTD-Plugin | 1 + contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 86 ++++++++ contrib/qatlib | 1 + contrib/qatlib-cmake/CMakeLists.txt | 197 +++++++++++++++++++ contrib/qatlib-cmake/include/mqueue.h | 41 ++++ contrib/qpl-cmake/CMakeLists.txt | 3 +- src/CMakeLists.txt | 4 + src/Compression/CompressionCodecZSTD.cpp | 96 ++++++++- src/Compression/CompressionFactory.cpp | 6 + 11 files changed, 461 insertions(+), 2 deletions(-) create mode 160000 contrib/QAT-ZSTD-Plugin create mode 100644 contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt create mode 160000 contrib/qatlib create mode 100644 contrib/qatlib-cmake/CMakeLists.txt create mode 100644 contrib/qatlib-cmake/include/mqueue.h diff --git a/.gitmodules b/.gitmodules index 53ef899dd99..cc26a1d0dce 100644 --- a/.gitmodules +++ b/.gitmodules @@ -360,3 +360,9 @@ [submodule "contrib/sqids-cpp"] path = contrib/sqids-cpp url = https://github.com/sqids/sqids-cpp.git +[submodule "contrib/QAT-ZSTD-Plugin"] + path = contrib/QAT-ZSTD-Plugin + url = https://github.com/intel/QAT-ZSTD-Plugin +[submodule "contrib/qatlib"] + path = contrib/qatlib + url = https://github.com/intel/qatlib diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 3d3b9b72faf..f847d04a63e 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -182,6 +182,28 @@ else() message(STATUS "Not using QPL") endif () +if (OS_LINUX AND ARCH_AMD64) + option (ENABLE_QAT "Enable Intel® QuickAssist Technology (Intel® QAT)" ${ENABLE_LIBRARIES}) +elseif(ENABLE_QAT) + message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64 arch") +endif() +if (ENABLE_QAT) + option (ENABLE_USDM_DRV "A User Space DMA-able Memory (USDM) component (kernel driver and corresponding user space library) which allocates/frees DMA-able memory" OFF) + option (ENABLE_QAT_OUTOFTREE "Using out-of-tree driver, user needs customize ICP_ROOT variable" OFF) + set(ICP_ROOT "" CACHE STRING "ICP_ROOT variable to define the path of out-of-tree driver package") +if (ENABLE_QAT_OUTOFTREE) +if(ICP_ROOT STREQUAL "") + message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUTOFTREE=OFF; \ + If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") +endif () +else() + add_contrib (qatlib-cmake qatlib) +endif () + add_contrib (QAT-ZSTD-Plugin-cmake QAT-ZSTD-Plugin) +else() + message(STATUS "Not using QAT") +endif () + add_contrib (morton-nd-cmake morton-nd) if (ARCH_S390X) add_contrib(crc32-s390x-cmake crc32-s390x) diff --git a/contrib/QAT-ZSTD-Plugin b/contrib/QAT-ZSTD-Plugin new file mode 160000 index 00000000000..e5a134e12d2 --- /dev/null +++ b/contrib/QAT-ZSTD-Plugin @@ -0,0 +1 @@ +Subproject commit e5a134e12d2ea8a5b0f3b83c5b1c325fda4eb0a8 diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt new file mode 100644 index 00000000000..db3b629965c --- /dev/null +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -0,0 +1,86 @@ +# Intel® QuickAssist Technology ZSTD Plugin (QAT ZSTD Plugin) is a plugin to Zstandard*(ZSTD*) for accelerating compression by QAT. + +if (ENABLE_QAT_OUTOFTREE) +# Intel QATZSTD out-of-tree build +message(STATUS "Intel QATZSTD out-of-tree build, ICP_ROOT:${ICP_ROOT}") + +set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") +set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") +set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") +set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") +set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") +set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") +set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") +set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") +set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") +if (ENABLE_USDM_DRV) + add_definitions(-DENABLE_USDM_DRV) +endif() +add_library(_qatzstd_plugin ${QATZSTD_SRC}) +target_link_libraries (_qatzstd_plugin PUBLIC ${USDM_LIBRARY} ${QAT_S_LIBRARY}) +target_include_directories(_qatzstd_plugin + SYSTEM PUBLIC "${QATZSTD_SRC_DIR}" + PRIVATE ${QAT_INCLUDE_DIR} + ${QAT_DC_INCLUDE_DIR} + ${QAT_AL_INCLUDE_DIR} + ${QAT_USDM_INCLUDE_DIR} + ${ZSTD_LIBRARY_DIR}) +target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) +add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) + +else() + +# Intel QATZSTD in-tree build +message(STATUS "Intel QATZSTD in-tree build") +set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") +set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") +set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") + +# please download&build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html +set(ICP_ROOT "${ClickHouse_SOURCE_DIR}/contrib/qatlib") +set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") +set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") +set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") +set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") +set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") +set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") +set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib") +set(LIBQAT_HEADER_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") + +file(MAKE_DIRECTORY + "${LIBQAT_HEADER_DIR}/qat" +) +file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/cpa.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" +) +file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/dc/cpa_dc.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" +) +file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_poll.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" +) +file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_user.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" +) +file(COPY "${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv/qae_mem.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" +) + +if (ENABLE_USDM_DRV) + add_definitions(-DENABLE_USDM_DRV) +endif() + +add_library(_qatzstd_plugin ${QATZSTD_SRC}) +target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qat ch_contrib::usdm) +target_include_directories(_qatzstd_plugin PRIVATE + ${QAT_INCLUDE_DIR} + ${QAT_DC_INCLUDE_DIR} + ${QAT_AL_INCLUDE_DIR} + ${QAT_USDM_INCLUDE_DIR} + ${ZSTD_LIBRARY_DIR} + ${LIBQAT_HEADER_DIR}) +target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) +add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) +target_include_directories(_qatzstd_plugin SYSTEM BEFORE PUBLIC ${QATZSTD_SRC_DIR}) +endif () + diff --git a/contrib/qatlib b/contrib/qatlib new file mode 160000 index 00000000000..abe15d7bfc0 --- /dev/null +++ b/contrib/qatlib @@ -0,0 +1 @@ +Subproject commit abe15d7bfc083117bfbb4baee0b49ffcd1c03c5c diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt new file mode 100644 index 00000000000..7671cd5913e --- /dev/null +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -0,0 +1,197 @@ +# Intel® QuickAssist Technology Library (QATlib). +message(STATUS "Intel QATlib ON") +set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib") +set(LIBQAT_DIR "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src") +set(LIBOSAL_DIR "${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src") +set(OPENSSL_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl") + +# Generate 3 library targets: _qatmgr, _osal, _qat, Output ch_contrib::qat by linking with these targets. +#_qatmgr +SET(LIBQATMGR_Sources ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_client.c + ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_lib.c + ${LIBQAT_DIR}/qat_direct/vfio/qat_log.c + ${LIBQAT_DIR}/qat_direct/vfio/vfio_lib.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_proto.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_vf_msg.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_pf.c) +add_library(_qatmgr ${LIBQATMGR_Sources}) +target_include_directories(_qatmgr PRIVATE + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/vfio + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include + ${LIBQAT_ROOT_DIR}/quickassist/include + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include + ${LIBQAT_ROOT_DIR}/quickassist/qat/drivers/crypto/qat/qat_common + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include + ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) +target_compile_definitions(_qatmgr PRIVATE -DUSER_SPACE) + +#_osal +SET(LIBOSAL_Sources + ${LIBOSAL_DIR}/linux/user_space/OsalSemaphore.c + ${LIBOSAL_DIR}/linux/user_space/OsalThread.c + ${LIBOSAL_DIR}/linux/user_space/OsalMutex.c + ${LIBOSAL_DIR}/linux/user_space/OsalSpinLock.c + ${LIBOSAL_DIR}/linux/user_space/OsalAtomic.c + ${LIBOSAL_DIR}/linux/user_space/OsalServices.c + ${LIBOSAL_DIR}/linux/user_space/OsalUsrKrnProxy.c + ${LIBOSAL_DIR}/linux/user_space/OsalCryptoInterface.c) +add_library(_osal ${LIBOSAL_Sources}) +target_include_directories(_osal PRIVATE + ${CMAKE_SYSROOT}/usr/include + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include + ${OPENSSL_DIR}/include + ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include + ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include) +target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) + +#_qat +SET(LIBQAT_Sources + ${LIBQAT_DIR}/common/compression/dc_buffers.c + ${LIBQAT_DIR}/common/compression/dc_chain.c + ${LIBQAT_DIR}/common/compression/dc_datapath.c + ${LIBQAT_DIR}/common/compression/dc_dp.c + ${LIBQAT_DIR}/common/compression/dc_header_footer.c + ${LIBQAT_DIR}/common/compression/dc_header_footer_lz4.c + ${LIBQAT_DIR}/common/compression/dc_session.c + ${LIBQAT_DIR}/common/compression/dc_stats.c + ${LIBQAT_DIR}/common/compression/dc_err_sim.c + ${LIBQAT_DIR}/common/compression/dc_ns_datapath.c + ${LIBQAT_DIR}/common/compression/dc_ns_header_footer.c + ${LIBQAT_DIR}/common/compression/dc_crc32.c + ${LIBQAT_DIR}/common/compression/dc_crc64.c + ${LIBQAT_DIR}/common/compression/dc_xxhash32.c + ${LIBQAT_DIR}/common/compression/icp_sal_dc_err_sim.c + ${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_control_path.c + ${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_data_path.c + ${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_interface_check.c + ${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_stats.c + ${LIBQAT_DIR}/common/crypto/asym/dsa/lac_dsa.c + ${LIBQAT_DIR}/common/crypto/asym/dsa/lac_dsa_interface_check.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_common.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_montedwds.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_nist_curves.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecdh.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecdsa.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecsm2.c + ${LIBQAT_DIR}/common/crypto/asym/ecc/lac_kpt_ecdsa.c + ${LIBQAT_DIR}/common/crypto/asym/large_number/lac_ln.c + ${LIBQAT_DIR}/common/crypto/asym/large_number/lac_ln_interface_check.c + ${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_mmp.c + ${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_qat_comms.c + ${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_utils.c + ${LIBQAT_DIR}/common/crypto/asym/prime/lac_prime.c + ${LIBQAT_DIR}/common/crypto/asym/prime/lac_prime_interface_check.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_control_path.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_decrypt.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_encrypt.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_interface_check.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_keygen.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_stats.c + ${LIBQAT_DIR}/common/crypto/asym/rsa/lac_kpt_rsa_decrypt.c + ${LIBQAT_DIR}/common/crypto/sym/drbg/lac_sym_drbg_api.c + ${LIBQAT_DIR}/common/crypto/sym/key/lac_sym_key.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_alg_chain.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_api.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_auth_enc.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_cb.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_cipher.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_compile_check.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_dp.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_hash.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_partial.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_queue.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_stats.c + ${LIBQAT_DIR}/common/crypto/sym/nrbg/lac_sym_nrbg_api.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_cipher.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_constants_table.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_hash.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_hash_defs_lookup.c + ${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_key.c + ${LIBQAT_DIR}/common/crypto/sym/lac_sym_hash_sw_precomputes.c + ${LIBQAT_DIR}/common/crypto/kpt/provision/lac_kpt_provision.c + ${LIBQAT_DIR}/common/ctrl/sal_compression.c + ${LIBQAT_DIR}/common/ctrl/sal_create_services.c + ${LIBQAT_DIR}/common/ctrl/sal_ctrl_services.c + ${LIBQAT_DIR}/common/ctrl/sal_list.c + ${LIBQAT_DIR}/common/ctrl/sal_crypto.c + ${LIBQAT_DIR}/common/ctrl/sal_dc_chain.c + ${LIBQAT_DIR}/common/ctrl/sal_instances.c + ${LIBQAT_DIR}/common/qat_comms/sal_qat_cmn_msg.c + ${LIBQAT_DIR}/common/utils/lac_buffer_desc.c + ${LIBQAT_DIR}/common/utils/lac_log_message.c + ${LIBQAT_DIR}/common/utils/lac_mem.c + ${LIBQAT_DIR}/common/utils/lac_mem_pools.c + ${LIBQAT_DIR}/common/utils/lac_sw_responses.c + ${LIBQAT_DIR}/common/utils/lac_sync.c + ${LIBQAT_DIR}/common/utils/sal_service_state.c + ${LIBQAT_DIR}/common/utils/sal_statistics.c + ${LIBQAT_DIR}/common/utils/sal_misc_error_stats.c + ${LIBQAT_DIR}/common/utils/sal_string_parse.c + ${LIBQAT_DIR}/common/utils/sal_user_process.c + ${LIBQAT_DIR}/common/utils/sal_versions.c + ${LIBQAT_DIR}/common/device/sal_dev_info.c + ${LIBQAT_DIR}/user/sal_user.c + ${LIBQAT_DIR}/user/sal_user_dyn_instance.c + ${LIBQAT_DIR}/qat_direct/common/adf_process_proxy.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_cfg.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_device.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_dyn.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_ETring_mgr_dp.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_init.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_ring.c + ${LIBQAT_DIR}/qat_direct/common/adf_user_transport_ctrl.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_cfg.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_ring.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_bundles.c + ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_proxy.c + ${LIBQAT_DIR}/common/compression/dc_crc_base.c) + +add_library(_qat ${LIBQAT_Sources}) +target_include_directories(_qat PRIVATE + ${CMAKE_SYSROOT}/usr/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include + ${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include + ${LIBOSAL_DIR}/linux/user_space/include + ${LIBQAT_ROOT_DIR}/quickassist/include + ${LIBQAT_ROOT_DIR}/quickassist/include/lac + ${LIBQAT_ROOT_DIR}/quickassist/include/dc + ${LIBQAT_ROOT_DIR}/quickassist/qat/drivers/crypto/qat/qat_common + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/compression/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/crypto/sym/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/crypto/asym/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/firmware/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include + ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/vfio + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space + ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include + ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) +target_link_libraries (_qat PRIVATE _qatmgr _osal OpenSSL::SSL) +target_compile_definitions(_qat PRIVATE -DUSER_SPACE -DLAC_BYTE_ORDER=__LITTLE_ENDIAN -DOSAL_ENSURE_ON -DUSE_CCODE_CRC) +target_link_options(_qat PRIVATE -pie -z relro -z now -z noexecstack) +target_compile_options(_qat PRIVATE -march=native) +add_library (ch_contrib::qat ALIAS _qat) + +#_usdm +set(LIBUSDM_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib/quickassist/utilities/libusdm_drv") +set(LIBUSDM_Sources + ${LIBUSDM_DIR}/user_space/vfio/qae_mem_utils_vfio.c + ${LIBUSDM_DIR}/user_space/qae_mem_utils_common.c + ${LIBUSDM_DIR}/user_space/vfio/qae_mem_hugepage_utils_vfio.c) +add_library(_usdm ${LIBUSDM_Sources}) + +target_include_directories(_usdm PRIVATE + ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include + ${LIBUSDM_DIR} + ${LIBUSDM_DIR}/include + ${LIBUSDM_DIR}/user_space) +add_library (ch_contrib::usdm ALIAS _usdm) \ No newline at end of file diff --git a/contrib/qatlib-cmake/include/mqueue.h b/contrib/qatlib-cmake/include/mqueue.h new file mode 100644 index 00000000000..c0f0f661b5d --- /dev/null +++ b/contrib/qatlib-cmake/include/mqueue.h @@ -0,0 +1,41 @@ +#ifndef _MQUEUE_H +#define _MQUEUE_H +#ifdef __cplusplus +extern "C" { +#endif + +#include + +#define __NEED_size_t +#define __NEED_ssize_t +#define __NEED_pthread_attr_t +#define __NEED_time_t +#define __NEED_struct_timespec +//#include + +typedef int mqd_t; +struct mq_attr { + long mq_flags, mq_maxmsg, mq_msgsize, mq_curmsgs, __unused[4]; +}; +struct sigevent; + +int mq_close(mqd_t); +int mq_getattr(mqd_t, struct mq_attr *); +int mq_notify(mqd_t, const struct sigevent *); +mqd_t mq_open(const char *, int, ...); +ssize_t mq_receive(mqd_t, char *, size_t, unsigned *); +int mq_send(mqd_t, const char *, size_t, unsigned); +int mq_setattr(mqd_t, const struct mq_attr *__restrict, struct mq_attr *__restrict); +ssize_t mq_timedreceive(mqd_t, char *__restrict, size_t, unsigned *__restrict, const struct timespec *__restrict); +int mq_timedsend(mqd_t, const char *, size_t, unsigned, const struct timespec *); +int mq_unlink(const char *); + +#if _REDIR_TIME64 +__REDIR(mq_timedreceive, __mq_timedreceive_time64); +__REDIR(mq_timedsend, __mq_timedsend_time64); +#endif + +#ifdef __cplusplus +} +#endif +#endif diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 7a84048e16b..246e78fbc5c 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -713,7 +713,8 @@ target_compile_definitions(_qpl target_link_libraries(_qpl PRIVATE ch_contrib::accel-config - PRIVATE ch_contrib::isal) + PRIVATE ch_contrib::isal + PRIVATE ch_contrib::qat) target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include" diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6063c701708..84cc8cc21ad 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -557,6 +557,10 @@ if (TARGET ch_contrib::accel-config) dbms_target_link_libraries(PUBLIC ch_contrib::accel-config) endif () +if (TARGET ch_contrib::qatzstd_plugin) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::qatzstd_plugin) +endif () + target_link_libraries(clickhouse_common_io PUBLIC boost::context) dbms_target_link_libraries(PUBLIC boost::context) diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index ec37ec6a7b5..7949038ff82 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -9,7 +9,9 @@ #include #include #include - +#include "qatseqprod.h" +#include +#include namespace DB { @@ -156,6 +158,98 @@ void registerCodecZSTD(CompressionCodecFactory & factory) }); } +#ifdef ENABLE_QATZSTD_COMPRESSION +class CompressionCodecQATZSTD : public CompressionCodecZSTD +{ +public: + static constexpr auto QATZSTD_SUPPORTED_MIN_LEVEL = 1; + static constexpr auto QATZSTD_SUPPORTED_MAX_LEVEL = 12; + explicit CompressionCodecQATZSTD(int level_); + ~CompressionCodecQATZSTD() override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + +private: + const int level; + mutable bool initialized; + mutable ZSTD_CCtx* cctx; + mutable void *sequenceProducerState; + Poco::Logger * log; +}; + +UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + if(!initialized) + { + cctx = ZSTD_createCCtx(); + /* Start QAT device, start QAT device at any time before compression job started */ + int res = QZSTD_startQatDevice(); + /* Create sequence producer state for QAT sequence producer */ + sequenceProducerState = QZSTD_createSeqProdState(); + /* register qatSequenceProducer */ + ZSTD_registerSequenceProducer( + cctx, + sequenceProducerState, + qatSequenceProducer + ); + /* Enable sequence producer fallback */ + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); + initialized = true; + LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); + } + size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); + + if (ZSTD_isError(compressed_size)) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); + + return static_cast(compressed_size); +} + +void registerCodecQATZSTD(CompressionCodecFactory & factory) +{ + factory.registerCompressionCodec("QATZSTD", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() > 1) + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "QATZSTD codec must have 1 parameter, given {}", arguments->children.size()); + + const auto children = arguments->children; + const auto * literal = children[0]->as(); + if (!literal) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "QATZSTD codec argument must be integer"); + + level = static_cast(literal->value.safeGet()); + if (level > CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL || level < CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, + "QATZSTD codec doesn't support level more than {} and lower than {} , given {}", + CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL, CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL, level); + } + + return std::make_shared(level); + }); +} + +CompressionCodecQATZSTD::CompressionCodecQATZSTD(int level_) + : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecQATZSTD")) +{ + setCodecDescription("QATZSTD", {std::make_shared(static_cast(level))}); +} + +CompressionCodecQATZSTD::~CompressionCodecQATZSTD() +{ + if(initialized) + { + /* Free sequence producer state */ + QZSTD_freeSeqProdState(sequenceProducerState); + ZSTD_freeCCtx(cctx); + } +} +#endif + CompressionCodecPtr getCompressionCodecZSTD(int level) { return std::make_shared(level); diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 7959c431328..96c245bb021 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -171,6 +171,9 @@ void registerCodecMultiple(CompressionCodecFactory & factory); #ifdef ENABLE_QPL_COMPRESSION void registerCodecDeflateQpl(CompressionCodecFactory & factory); #endif +#ifdef ENABLE_QATZSTD_COMPRESSION +void registerCodecQATZSTD(CompressionCodecFactory & factory); +#endif /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build @@ -200,6 +203,9 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecFPC(*this); #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflateQpl(*this); +#endif +#ifdef ENABLE_QATZSTD_COMPRESSION + registerCodecQATZSTD(*this); #endif registerCodecGCD(*this); #endif From c1c46b655fd2e639438d4a9415c331e88fe15227 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Dec 2023 18:02:05 -0500 Subject: [PATCH 008/116] add build flag to avoid incompatible integer conversion --- contrib/qatlib-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 7671cd5913e..8e23c1466ca 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -26,6 +26,7 @@ target_include_directories(_qatmgr PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) target_compile_definitions(_qatmgr PRIVATE -DUSER_SPACE) +target_compile_options(_qatmgr PRIVATE -Wno-error=int-conversion) #_osal SET(LIBOSAL_Sources From da10b21616121ef32a265769e28f04e9c94d8a40 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Fri, 8 Dec 2023 17:42:11 +0000 Subject: [PATCH 009/116] Some test clarification --- .../02896_leading_zeroes_no_octal.reference | 10 ++++++++++ .../0_stateless/02896_leading_zeroes_no_octal.sql | 10 ++++++++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference index 69446796e09..5b932f50824 100644 --- a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.reference @@ -23,3 +23,13 @@ Leading zeroes into Float32 1 3 00009e9 9000000000 9000000000 1 4 00009e09 9000000000 9000000000 1 5 00009e0009 9000000000 9000000000 +1 6 -00000 -0.1 -0.1 +1 7 -00009.00009 -9.00009 -9.00009 +1 8 -00009e9 -9000000000 -9000000000 +1 9 -00009e09 -9000000000 -9000000000 +1 10 -00009e0009 -9000000000 -9000000000 +1 11 +00000 0 0 +1 12 +00009.00009 9.00009 9.00009 +1 13 +00009e9 9000000000 9000000000 +1 14 +00009e09 9000000000 9000000000 +1 15 +00009e0009 9000000000 9000000000 diff --git a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql index a204259f3ec..69cc06a46f8 100644 --- a/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql +++ b/tests/queries/0_stateless/02896_leading_zeroes_no_octal.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS t_leading_zeroes; +DROP TABLE IF EXISTS t_leading_zeroes_f; + CREATE TABLE t_leading_zeroes(id INTEGER, input String, val INTEGER, expected INTEGER) ENGINE=MergeTree ORDER BY id; INSERT INTO t_leading_zeroes VALUES (1, '00000', 00000, 0), (2, '0', 0, 0), (3, '00', 00, 0), (4, '01', 01, 1), (5, '+01', +01, 1); @@ -10,10 +12,14 @@ INSERT INTO t_leading_zeroes VALUES (17, '1000.0008', 1000.0008, 1000), (18, '00 SELECT 'Leading zeroes into INTEGER'; SELECT t.val == t.expected AS ok, * FROM t_leading_zeroes t ORDER BY id; +-- Floats don't go via the weird octal path: CREATE TABLE t_leading_zeroes_f(id INTEGER, input String, val Float32, expected Float32) ENGINE=MergeTree ORDER BY id; INSERT INTO t_leading_zeroes_f VALUES (1, '00000', 00000, 0), (2, '00009.00009', 00009.00009, 9.00009), (3, '00009e9', 00009e9, 9e9), (4, '00009e09', 00009e09, 9e9), (5, '00009e0009', 00009e0009, 9e9); --- Turns out this is not ok in master as well - will have a look and fix --- (6, '00009e00009', 00009e00009, 9e9); +INSERT INTO t_leading_zeroes_f VALUES (6, '-00000', -00000.1, -0.1), (7, '-00009.00009', -00009.00009, -9.00009), (8, '-00009e9', -00009e9, -9e9), (9, '-00009e09', -00009e09, -9e9), (10, '-00009e0009', -00009e0009, -9e9); +INSERT INTO t_leading_zeroes_f VALUES (11, '+00000', +00000., 0), (12, '+00009.00009', +00009.00009, 9.00009), (13, '+00009e9', +00009e9, 9e9), (14, '+00009e09', +00009e09, 9e9), (15, '+00009e0009', +00009e0009, 9e9); +-- Coincidentally, the following result in 9 rather than 9e9 because of readFloatTextFastImpl +-- using readUIntTextUpToNSignificantDigits<4>(exponent, in) +-- INSERT INTO t_leading_zeroes_f VALUES (100, '00009e00009', 00009e00009, 9e9), (101, '-00009e00009', -00009e00009, -9e9), (102, '+00009e00009', +00009e00009, 9e9) SELECT 'Leading zeroes into Float32'; SELECT t.val == t.expected AS ok, * FROM t_leading_zeroes_f t ORDER BY id; From 1e7407f82c03c38caec5618058cc399afc0f3fde Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 8 Dec 2023 20:14:55 +0000 Subject: [PATCH 010/116] Customazable dependency failure handling for AsyncLoader --- src/Common/AsyncLoader.cpp | 130 ++++++++++++++---------- src/Common/AsyncLoader.h | 70 +++++++++---- src/Common/tests/gtest_async_loader.cpp | 95 +++++++++++++++++ src/Databases/DatabaseOrdinary.cpp | 10 +- 4 files changed, 229 insertions(+), 76 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 962adb8b052..7c7b057143b 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -43,6 +44,19 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic } } +void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel) +{ + cancel = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> {}", + self->name, + getExceptionMessage(dependency->exception(), /* with_stacktrace = */ false))); +} + +void ignoreDependencyFailure(const LoadJobPtr &, const LoadJobPtr &, std::exception_ptr &) +{ + // No-op +} + LoadStatus LoadJob::status() const { std::unique_lock lock{mutex}; @@ -96,7 +110,10 @@ size_t LoadJob::canceled(const std::exception_ptr & ptr) size_t LoadJob::finish() { - func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` return + // To ensure functions are destructed before `AsyncLoader::wait()` return + func = {}; + dependency_failure = {}; + finish_time = std::chrono::system_clock::now(); if (waiters > 0) finished.notify_all(); @@ -327,17 +344,19 @@ void AsyncLoader::schedule(const LoadJobSet & jobs_to_schedule) if (dep_status == LoadStatus::FAILED || dep_status == LoadStatus::CANCELED) { - // Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs. - std::exception_ptr e; + // Dependency on already failed or canceled job -- it's okay. + // Process as usual (may lead to cancel of all dependent jobs). + std::exception_ptr cancel; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' -> {}", - job->name, - getExceptionMessage(dep->exception(), /* with_stacktrace = */ false))); + if (job->dependency_failure) + job->dependency_failure(job, dep, cancel); }); - finish(job, LoadStatus::CANCELED, e, lock); - break; // This job is now finished, stop its dependencies processing + if (cancel) + { + finish(job, LoadStatus::CANCELED, cancel, lock); + break; // This job is now finished, stop its dependencies processing + } } } } @@ -515,63 +534,69 @@ String AsyncLoader::checkCycle(const LoadJobPtr & job, LoadJobSet & left, LoadJo return {}; } -void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock) +void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr reason, std::unique_lock & lock) { chassert(scheduled_jobs.contains(job)); // Job was pending + + // Notify waiters size_t resumed_workers = 0; // Number of workers resumed in the execution pool of the job if (status == LoadStatus::OK) - { - // Notify waiters - resumed_workers += job->ok(); + resumed_workers = job->ok(); + else if (status == LoadStatus::FAILED) + resumed_workers = job->failed(reason); + else if (status == LoadStatus::CANCELED) + resumed_workers = job->canceled(reason); - // Update dependent jobs and enqueue if ready - for (const auto & dep : scheduled_jobs[job].dependent_jobs) - { - chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending - Info & dep_info = scheduled_jobs[dep]; - dep_info.dependencies_left--; - if (!dep_info.isBlocked()) - enqueue(dep_info, dep, lock); - } + // Adjust suspended workers count + if (resumed_workers) + { + Pool & pool = pools[job->executionPool()]; + pool.suspended_workers -= resumed_workers; } - else + + Info & info = scheduled_jobs[job]; + if (info.isReady()) { - // Notify waiters - if (status == LoadStatus::FAILED) - resumed_workers += job->failed(exception_from_job); - else if (status == LoadStatus::CANCELED) - resumed_workers += job->canceled(exception_from_job); + // Job could be in ready queue (on cancel) -- must be dequeued + pools[job->pool_id].ready_queue.erase(info.ready_seqno); + info.ready_seqno = 0; + } - Info & info = scheduled_jobs[job]; - if (info.isReady()) - { - pools[job->pool_id].ready_queue.erase(info.ready_seqno); - info.ready_seqno = 0; - } + // To avoid container modification during recursion (during clean dependency graph edges below) + LoadJobSet dependent; + dependent.swap(info.dependent_jobs); - // Recurse into all dependent jobs - LoadJobSet dependent; - dependent.swap(info.dependent_jobs); // To avoid container modification during recursion - for (const auto & dep : dependent) + // Update dependent jobs + for (const auto & dpt : dependent) + { + chassert(scheduled_jobs.contains(dpt)); // All dependent jobs must be scheduled + Info & dpt_info = scheduled_jobs[dpt]; + dpt_info.dependencies_left--; + if (!dpt_info.isBlocked()) + enqueue(dpt_info, dpt, lock); + + if (status != LoadStatus::OK) { - if (!scheduled_jobs.contains(dep)) - continue; // Job has already been canceled - std::exception_ptr e; + std::exception_ptr cancel; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' -> {}", - dep->name, - getExceptionMessage(exception_from_job, /* with_stacktrace = */ false))); + if (dpt->dependency_failure) + dpt->dependency_failure(dpt, job, cancel); }); - finish(dep, LoadStatus::CANCELED, e, lock); + // Recurse into dependent job if it should be canceled + if (cancel) + finish(dpt, LoadStatus::CANCELED, cancel, lock); } + } - // Clean dependency graph edges pointing to canceled jobs + // Clean dependency graph edges pointing to canceled jobs + if (status != LoadStatus::OK) + { for (const auto & dep : job->dependencies) + { if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) dep_info->second.dependent_jobs.erase(job); + } } // Job became finished @@ -582,12 +607,6 @@ void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::excepti if (log_progress) logAboutProgress(log, finished_jobs.size() - old_jobs, finished_jobs.size() + scheduled_jobs.size() - old_jobs, stopwatch); }); - - if (resumed_workers) - { - Pool & pool = pools[job->executionPool()]; - pool.suspended_workers -= resumed_workers; - } } void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock) @@ -612,6 +631,9 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un } job->pool_id.store(new_pool_id); + // TODO(serxa): we should adjust suspended_workers and suspended_waiters here. + // Otherwise suspended_workers we be left inconsistent. Fix it and add a test. + // Scenario: schedule a job A, wait for it from a job B in the same pool, prioritize A // Recurse into dependencies for (const auto & dep : job->dependencies) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 7b6e3ebfefe..fff36d9863a 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -8,6 +9,7 @@ #include #include #include +#include <__concepts/invocable.h> #include #include #include @@ -57,12 +59,13 @@ enum class LoadStatus class LoadJob : private boost::noncopyable { public: - template - LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, Func && func_) + template + LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, DFFunc && dependency_failure_, Func && func_) : dependencies(std::forward(dependencies_)) , name(std::move(name_)) , execution_pool_id(pool_id_) , pool_id(pool_id_) + , dependency_failure(std::forward(dependency_failure_)) , func(std::forward(func_)) {} @@ -108,6 +111,14 @@ private: std::atomic job_id{0}; std::atomic execution_pool_id; std::atomic pool_id; + + // Handler for failed or canceled dependencies. + // If job needs to be canceled on `dependency` failure, then function should set `cancel` to a specific reason. + // Note that implementation should be fast and cannot use AsyncLoader, because it is called under `AsyncLoader::mutex`. + // Note that `dependency_failure` is called only on pending jobs. + std::function dependency_failure; + + // Function to be called to execute the job. std::function func; mutable std::mutex mutex; @@ -123,35 +134,54 @@ private: std::atomic finish_time{TimePoint{}}; }; -struct EmptyJobFunc -{ - void operator()(AsyncLoader &, const LoadJobPtr &) {} -}; +// For LoadJob::dependency_failure. Cancels the job on the first dependency failure or cancel. +void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel); -template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func = EmptyJobFunc()) +// For LoadJob::dependency_failure. Never cancels the job due to dependency failure or cancel. +void ignoreDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel); + +template concept LoadJobDependencyFailure = std::invocable; +template concept LoadJobFunc = std::invocable; + +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) { - return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(func)); + return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(dependency_failure), std::forward(func)); } -template -LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) { - return std::make_shared(dependencies, std::move(name), 0, std::forward(func)); + return std::make_shared(dependencies, std::move(name), 0, std::forward(dependency_failure), std::forward(func)); } -template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) { - return std::make_shared(std::move(dependencies), std::move(name), pool_id, std::forward(func)); + return std::make_shared(std::move(dependencies), std::move(name), pool_id, std::forward(dependency_failure), std::forward(func)); } -template -LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) { - return std::make_shared(dependencies, std::move(name), pool_id, std::forward(func)); + return std::make_shared(dependencies, std::move(name), pool_id, std::forward(dependency_failure), std::forward(func)); } +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), 0, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), 0, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), pool_id, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), pool_id, cancelOnDependencyFailure, std::forward(func)); +} // Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set). class LoadTask : private boost::noncopyable @@ -277,7 +307,7 @@ private: { size_t dependencies_left = 0; // Current number of dependencies on pending jobs. UInt64 ready_seqno = 0; // Zero means that job is not in ready queue. - LoadJobSet dependent_jobs; // Set of jobs dependent on this job. + LoadJobSet dependent_jobs; // Set of jobs dependent on this job. Contains only scheduled jobs. // Three independent states of a scheduled job. bool isBlocked() const { return dependencies_left > 0; } @@ -371,7 +401,7 @@ public: private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); String checkCycle(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); - void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock); + void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr reason, std::unique_lock & lock); void gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, std::unique_lock & lock); void prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock); void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index f2e741e4b9f..4a332079e20 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -544,6 +546,99 @@ TEST(AsyncLoader, ScheduleJobWithCanceledDependencies) } } +TEST(AsyncLoader, IgnoreDependencyFailure) +{ + AsyncLoaderTest t; + std::atomic success{false}; + t.loader.start(); + + std::string_view error_message = "test job failure"; + + auto failed_job_func = [&] (AsyncLoader &, const LoadJobPtr &) { + throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message); + }; + auto dependent_job_func = [&] (AsyncLoader &, const LoadJobPtr &) { + success.store(true); + }; + + auto failed_job = makeLoadJob({}, "failed_job", failed_job_func); + auto dependent_job = makeLoadJob({failed_job}, + "dependent_job", ignoreDependencyFailure, dependent_job_func); + auto task = t.schedule({ failed_job, dependent_job }); + + t.loader.wait(); + + ASSERT_EQ(failed_job->status(), LoadStatus::FAILED); + ASSERT_EQ(dependent_job->status(), LoadStatus::OK); + ASSERT_EQ(success.load(), true); +} + +TEST(AsyncLoader, CustomDependencyFailure) +{ + AsyncLoaderTest t(16); + int error_count = 0; + std::atomic good_count{0}; + std::barrier canceled_sync(4); + t.loader.start(); + + std::string_view error_message = "test job failure"; + + auto evil_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) { + throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message); + }; + auto good_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) { + good_count++; + }; + auto late_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) { + canceled_sync.arrive_and_wait(); // wait for fail (A) before this job is finished + }; + auto collect_job_func = [&] (AsyncLoader &, const LoadJobPtr &) { + FAIL(); // job should be canceled, so we never get here + }; + auto dependent_job_func = [&] (AsyncLoader &, const LoadJobPtr &) { + FAIL(); // job should be canceled, so we never get here + }; + auto fail_after_two = [&] (const LoadJobPtr & self, const LoadJobPtr &, std::exception_ptr & cancel) { + if (++error_count == 2) + cancel = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' canceled: too many dependencies have failed", + self->name)); + }; + + auto evil_dep1 = makeLoadJob({}, "evil_dep1", evil_dep_func); + auto evil_dep2 = makeLoadJob({}, "evil_dep2", evil_dep_func); + auto evil_dep3 = makeLoadJob({}, "evil_dep3", evil_dep_func); + auto good_dep1 = makeLoadJob({}, "good_dep1", good_dep_func); + auto good_dep2 = makeLoadJob({}, "good_dep2", good_dep_func); + auto good_dep3 = makeLoadJob({}, "good_dep3", good_dep_func); + auto late_dep1 = makeLoadJob({}, "late_dep1", late_dep_func); + auto late_dep2 = makeLoadJob({}, "late_dep2", late_dep_func); + auto late_dep3 = makeLoadJob({}, "late_dep3", late_dep_func); + auto collect_job = makeLoadJob({ + evil_dep1, evil_dep2, evil_dep3, + good_dep1, good_dep2, good_dep3, + late_dep1, late_dep2, late_dep3 + }, "collect_job", fail_after_two, collect_job_func); + auto dependent_job1 = makeLoadJob({ collect_job }, "dependent_job1", dependent_job_func); + auto dependent_job2 = makeLoadJob({ collect_job }, "dependent_job2", dependent_job_func); + auto dependent_job3 = makeLoadJob({ collect_job }, "dependent_job3", dependent_job_func); + auto task = t.schedule({ dependent_job1, dependent_job2, dependent_job3 }); // Other jobs should be discovery automatically + + t.loader.wait(collect_job, true); + canceled_sync.arrive_and_wait(); // (A) + + t.loader.wait(); + + ASSERT_EQ(late_dep1->status(), LoadStatus::OK); + ASSERT_EQ(late_dep2->status(), LoadStatus::OK); + ASSERT_EQ(late_dep3->status(), LoadStatus::OK); + ASSERT_EQ(collect_job->status(), LoadStatus::CANCELED); + ASSERT_EQ(dependent_job1->status(), LoadStatus::CANCELED); + ASSERT_EQ(dependent_job2->status(), LoadStatus::CANCELED); + ASSERT_EQ(dependent_job3->status(), LoadStatus::CANCELED); + ASSERT_EQ(good_count.load(), 3); +} + TEST(AsyncLoader, TestConcurrency) { AsyncLoaderTest t(10); diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 1f344551c5e..e3b8b03f35a 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -20,6 +20,7 @@ #include #include #include +#include "Common/AsyncLoader.h" #include #include #include @@ -223,11 +224,16 @@ LoadTaskPtr DatabaseOrdinary::startupDatabaseAsync( LoadJobSet startup_after, LoadingStrictnessLevel /*mode*/) { - // NOTE: this task is empty, but it is required for correct dependency handling (startup should be done after tables loading) auto job = makeLoadJob( std::move(startup_after), TablesLoaderBackgroundStartupPoolId, - fmt::format("startup Ordinary database {}", getDatabaseName())); + fmt::format("startup Ordinary database {}", getDatabaseName()), + ignoreDependencyFailure, + [] (AsyncLoader &, const LoadJobPtr &) { + // NOTE: this job is no-op, but it is required for correct dependency handling + // 1) startup should be done after tables loading + // 2) load or startup errors for tables should not lead to not starting up the whole database + }); return startup_database_task = makeLoadTask(async_loader, {job}); } From 58138861bab0d171cdfff961dc645b75fb86cf15 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 8 Dec 2023 22:34:47 +0000 Subject: [PATCH 011/116] fix style --- src/Databases/DatabaseOrdinary.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index e3b8b03f35a..d9e2c1f39da 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -229,7 +229,8 @@ LoadTaskPtr DatabaseOrdinary::startupDatabaseAsync( TablesLoaderBackgroundStartupPoolId, fmt::format("startup Ordinary database {}", getDatabaseName()), ignoreDependencyFailure, - [] (AsyncLoader &, const LoadJobPtr &) { + [] (AsyncLoader &, const LoadJobPtr &) + { // NOTE: this job is no-op, but it is required for correct dependency handling // 1) startup should be done after tables loading // 2) load or startup errors for tables should not lead to not starting up the whole database From a25de0cc64d395be8c082ad4f6c8caef02f2fcc8 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Dec 2023 09:57:41 -0500 Subject: [PATCH 012/116] refine cosmos issue --- src/Compression/CompressionCodecZSTD.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 7949038ff82..352941d00a2 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -180,7 +180,7 @@ private: UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const { - if(!initialized) + if (!initialized) { cctx = ZSTD_createCCtx(); /* Start QAT device, start QAT device at any time before compression job started */ @@ -241,7 +241,7 @@ CompressionCodecQATZSTD::CompressionCodecQATZSTD(int level_) CompressionCodecQATZSTD::~CompressionCodecQATZSTD() { - if(initialized) + if (initialized) { /* Free sequence producer state */ QZSTD_freeSeqProdState(sequenceProducerState); From 4ba36f2761a00144e192a59be612cf549bb2279c Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Dec 2023 13:31:52 -0500 Subject: [PATCH 013/116] tried to fix missing header --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 84cc8cc21ad..164cc3337df 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -558,6 +558,7 @@ if (TARGET ch_contrib::accel-config) endif () if (TARGET ch_contrib::qatzstd_plugin) + dbms_target_link_libraries(PUBLIC ch_contrib::qatzstd_plugin) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::qatzstd_plugin) endif () From 8b6d6d52d4398213d34a3d95e160fde0a717f7b5 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Dec 2023 16:37:49 -0500 Subject: [PATCH 014/116] refine cmake for header --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index db3b629965c..3fd91b13181 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -81,6 +81,7 @@ target_include_directories(_qatzstd_plugin PRIVATE ${LIBQAT_HEADER_DIR}) target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) -target_include_directories(_qatzstd_plugin SYSTEM BEFORE PUBLIC ${QATZSTD_SRC_DIR}) +target_include_directories(_qatzstd_plugin SYSTEM PUBLIC ${QATZSTD_SRC_DIR}) +#target_compile_options(_qatzstd_plugin PRIVATE -Wno-zero-as-null-pointer-constant) endif () From f90953ccc47967958cdadfa4b864d6fb0ba2294e Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Dec 2023 17:54:39 -0500 Subject: [PATCH 015/116] refine cmake for header build --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 3fd91b13181..b24fadda20d 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -81,7 +81,6 @@ target_include_directories(_qatzstd_plugin PRIVATE ${LIBQAT_HEADER_DIR}) target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) -target_include_directories(_qatzstd_plugin SYSTEM PUBLIC ${QATZSTD_SRC_DIR}) -#target_compile_options(_qatzstd_plugin PRIVATE -Wno-zero-as-null-pointer-constant) +target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) endif () From 969c63df834245eff566e5ebd324408764b341a0 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Dec 2023 17:56:03 -0500 Subject: [PATCH 016/116] refine cmake for header issue --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index b24fadda20d..d41cb619a82 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -80,7 +80,7 @@ target_include_directories(_qatzstd_plugin PRIVATE ${ZSTD_LIBRARY_DIR} ${LIBQAT_HEADER_DIR}) target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) -add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) +add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) endif () From f231074c9bb8d38c4246c1900cab5d85184a0210 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 08:43:31 -0500 Subject: [PATCH 017/116] add macro for header file --- src/Compression/CompressionCodecZSTD.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 352941d00a2..7eb7aaf3222 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -9,9 +9,11 @@ #include #include #include -#include "qatseqprod.h" #include #include +#ifdef ENABLE_QATZSTD_COMPRESSION +#include "qatseqprod.h" +#endif namespace DB { From 3771acdefbb441059ff01e9e779be3e6bd31c2a9 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 11:15:11 -0500 Subject: [PATCH 018/116] fixed reported code issues --- .gitmodules | 12 +- contrib/CMakeLists.txt | 10 +- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 8 +- contrib/qatlib-cmake/CMakeLists.txt | 26 ++-- contrib/qatlib-cmake/include/mqueue.h | 41 ------ contrib/qpl-cmake/CMakeLists.txt | 3 +- src/CMakeLists.txt | 2 +- src/Compression/CompressionCodecQATZSTD.cpp | 114 ++++++++++++++++ src/Compression/CompressionCodecZSTD.cpp | 129 +------------------ src/Compression/CompressionCodecZSTD.h | 36 ++++++ 10 files changed, 183 insertions(+), 198 deletions(-) create mode 100644 src/Compression/CompressionCodecQATZSTD.cpp create mode 100644 src/Compression/CompressionCodecZSTD.h diff --git a/.gitmodules b/.gitmodules index cc26a1d0dce..2be300c04d0 100644 --- a/.gitmodules +++ b/.gitmodules @@ -245,6 +245,12 @@ [submodule "contrib/idxd-config"] path = contrib/idxd-config url = https://github.com/intel/idxd-config +[submodule "contrib/QAT-ZSTD-Plugin"] + path = contrib/QAT-ZSTD-Plugin + url = https://github.com/intel/QAT-ZSTD-Plugin +[submodule "contrib/qatlib"] + path = contrib/qatlib + url = https://github.com/intel/qatlib [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash @@ -360,9 +366,3 @@ [submodule "contrib/sqids-cpp"] path = contrib/sqids-cpp url = https://github.com/sqids/sqids-cpp.git -[submodule "contrib/QAT-ZSTD-Plugin"] - path = contrib/QAT-ZSTD-Plugin - url = https://github.com/intel/QAT-ZSTD-Plugin -[submodule "contrib/qatlib"] - path = contrib/qatlib - url = https://github.com/intel/qatlib diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f847d04a63e..05f308508d4 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -183,16 +183,16 @@ else() endif () if (OS_LINUX AND ARCH_AMD64) - option (ENABLE_QAT "Enable Intel® QuickAssist Technology (Intel® QAT)" ${ENABLE_LIBRARIES}) -elseif(ENABLE_QAT) + option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology (Intel® QATlib)" ${ENABLE_LIBRARIES}) +elseif(ENABLE_QATLIB) message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64 arch") endif() -if (ENABLE_QAT) - option (ENABLE_USDM_DRV "A User Space DMA-able Memory (USDM) component (kernel driver and corresponding user space library) which allocates/frees DMA-able memory" OFF) +if (ENABLE_QATLIB) + option (ENABLE_USDM_DRV "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF) option (ENABLE_QAT_OUTOFTREE "Using out-of-tree driver, user needs customize ICP_ROOT variable" OFF) set(ICP_ROOT "" CACHE STRING "ICP_ROOT variable to define the path of out-of-tree driver package") if (ENABLE_QAT_OUTOFTREE) -if(ICP_ROOT STREQUAL "") +if (ICP_ROOT STREQUAL "") message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUTOFTREE=OFF; \ If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") endif () diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index d41cb619a82..edc41866edd 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -1,4 +1,7 @@ # Intel® QuickAssist Technology ZSTD Plugin (QAT ZSTD Plugin) is a plugin to Zstandard*(ZSTD*) for accelerating compression by QAT. +# Out-of-tree means kernel don't have native support, user will build and install driver from external package: https://www.intel.com/content/www/us/en/download/765501.html +# meanwhile, user need to set ICP_ROOT environment variable which point to the root directory of QAT driver source tree. +# in-tree means kernel has built-in qat driver, QAT-ZSTD-PLUGIN just has dependency on qatlib. if (ENABLE_QAT_OUTOFTREE) # Intel QATZSTD out-of-tree build @@ -28,7 +31,7 @@ target_include_directories(_qatzstd_plugin target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) -else() +else () # Intel QATZSTD in-tree build message(STATUS "Intel QATZSTD in-tree build") @@ -71,7 +74,7 @@ if (ENABLE_USDM_DRV) endif() add_library(_qatzstd_plugin ${QATZSTD_SRC}) -target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qat ch_contrib::usdm) +target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qatlib ch_contrib::usdm) target_include_directories(_qatzstd_plugin PRIVATE ${QAT_INCLUDE_DIR} ${QAT_DC_INCLUDE_DIR} @@ -82,5 +85,6 @@ target_include_directories(_qatzstd_plugin PRIVATE target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) + endif () diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 8e23c1466ca..88d0cd4aa9e 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -5,7 +5,7 @@ set(LIBQAT_DIR "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src") set(LIBOSAL_DIR "${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src") set(OPENSSL_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl") -# Generate 3 library targets: _qatmgr, _osal, _qat, Output ch_contrib::qat by linking with these targets. +# Generate 3 library targets: _qatmgr, _osal, _qatlib, Output ch_contrib::qatlib by linking with these targets. #_qatmgr SET(LIBQATMGR_Sources ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_client.c ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_lib.c @@ -40,16 +40,16 @@ SET(LIBOSAL_Sources ${LIBOSAL_DIR}/linux/user_space/OsalCryptoInterface.c) add_library(_osal ${LIBOSAL_Sources}) target_include_directories(_osal PRIVATE - ${CMAKE_SYSROOT}/usr/include + ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/include ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include ${OPENSSL_DIR}/include - ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include - ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include) -target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) + ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include + ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include) +target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL -D__GLIBC_PREREQ=0) -#_qat +#_qatlib SET(LIBQAT_Sources ${LIBQAT_DIR}/common/compression/dc_buffers.c ${LIBQAT_DIR}/common/compression/dc_chain.c @@ -154,8 +154,8 @@ SET(LIBQAT_Sources ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_proxy.c ${LIBQAT_DIR}/common/compression/dc_crc_base.c) -add_library(_qat ${LIBQAT_Sources}) -target_include_directories(_qat PRIVATE +add_library(_qatlib ${LIBQAT_Sources}) +target_include_directories(_qatlib PRIVATE ${CMAKE_SYSROOT}/usr/include ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include ${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv @@ -176,11 +176,11 @@ target_include_directories(_qat PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) -target_link_libraries (_qat PRIVATE _qatmgr _osal OpenSSL::SSL) -target_compile_definitions(_qat PRIVATE -DUSER_SPACE -DLAC_BYTE_ORDER=__LITTLE_ENDIAN -DOSAL_ENSURE_ON -DUSE_CCODE_CRC) -target_link_options(_qat PRIVATE -pie -z relro -z now -z noexecstack) -target_compile_options(_qat PRIVATE -march=native) -add_library (ch_contrib::qat ALIAS _qat) +target_link_libraries(_qatlib PRIVATE _qatmgr _osal OpenSSL::SSL ch_contrib::isal) +target_compile_definitions(_qatlib PRIVATE -DUSER_SPACE -DLAC_BYTE_ORDER=__LITTLE_ENDIAN -DOSAL_ENSURE_ON) +target_link_options(_qatlib PRIVATE -pie -z relro -z now -z noexecstack) +target_compile_options(_qatlib PRIVATE -march=native) +add_library (ch_contrib::qatlib ALIAS _qatlib) #_usdm set(LIBUSDM_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib/quickassist/utilities/libusdm_drv") diff --git a/contrib/qatlib-cmake/include/mqueue.h b/contrib/qatlib-cmake/include/mqueue.h index c0f0f661b5d..e69de29bb2d 100644 --- a/contrib/qatlib-cmake/include/mqueue.h +++ b/contrib/qatlib-cmake/include/mqueue.h @@ -1,41 +0,0 @@ -#ifndef _MQUEUE_H -#define _MQUEUE_H -#ifdef __cplusplus -extern "C" { -#endif - -#include - -#define __NEED_size_t -#define __NEED_ssize_t -#define __NEED_pthread_attr_t -#define __NEED_time_t -#define __NEED_struct_timespec -//#include - -typedef int mqd_t; -struct mq_attr { - long mq_flags, mq_maxmsg, mq_msgsize, mq_curmsgs, __unused[4]; -}; -struct sigevent; - -int mq_close(mqd_t); -int mq_getattr(mqd_t, struct mq_attr *); -int mq_notify(mqd_t, const struct sigevent *); -mqd_t mq_open(const char *, int, ...); -ssize_t mq_receive(mqd_t, char *, size_t, unsigned *); -int mq_send(mqd_t, const char *, size_t, unsigned); -int mq_setattr(mqd_t, const struct mq_attr *__restrict, struct mq_attr *__restrict); -ssize_t mq_timedreceive(mqd_t, char *__restrict, size_t, unsigned *__restrict, const struct timespec *__restrict); -int mq_timedsend(mqd_t, const char *, size_t, unsigned, const struct timespec *); -int mq_unlink(const char *); - -#if _REDIR_TIME64 -__REDIR(mq_timedreceive, __mq_timedreceive_time64); -__REDIR(mq_timedsend, __mq_timedsend_time64); -#endif - -#ifdef __cplusplus -} -#endif -#endif diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 246e78fbc5c..7a84048e16b 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -713,8 +713,7 @@ target_compile_definitions(_qpl target_link_libraries(_qpl PRIVATE ch_contrib::accel-config - PRIVATE ch_contrib::isal - PRIVATE ch_contrib::qat) + PRIVATE ch_contrib::isal) target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include" diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 164cc3337df..3d94c4ab3e4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -550,7 +550,7 @@ endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) if (TARGET ch_contrib::qpl) -dbms_target_link_libraries(PUBLIC ch_contrib::qpl) + dbms_target_link_libraries(PUBLIC ch_contrib::qpl) endif () if (TARGET ch_contrib::accel-config) diff --git a/src/Compression/CompressionCodecQATZSTD.cpp b/src/Compression/CompressionCodecQATZSTD.cpp new file mode 100644 index 00000000000..eca1adc6f25 --- /dev/null +++ b/src/Compression/CompressionCodecQATZSTD.cpp @@ -0,0 +1,114 @@ +#ifdef ENABLE_QATZSTD_COMPRESSION +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "qatseqprod.h" +#include "CompressionCodecZSTD.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +class CompressionCodecQATZSTD : public CompressionCodecZSTD +{ +public: + static constexpr auto QATZSTD_SUPPORTED_MIN_LEVEL = 1; + static constexpr auto QATZSTD_SUPPORTED_MAX_LEVEL = 12; + explicit CompressionCodecQATZSTD(int level_); + ~CompressionCodecQATZSTD() override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + +private: + const int level; + mutable bool initialized; + mutable ZSTD_CCtx* cctx; + mutable void *sequenceProducerState; + Poco::Logger * log; +}; + +UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + if (!initialized) + { + cctx = ZSTD_createCCtx(); + /* Start QAT device, start QAT device at any time before compression job started */ + int res = QZSTD_startQatDevice(); + /* Create sequence producer state for QAT sequence producer */ + sequenceProducerState = QZSTD_createSeqProdState(); + /* register qatSequenceProducer */ + ZSTD_registerSequenceProducer( + cctx, + sequenceProducerState, + qatSequenceProducer + ); + /* Enable sequence producer fallback */ + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); + initialized = true; + LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); + } + size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); + + if (ZSTD_isError(compressed_size)) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); + + return static_cast(compressed_size); +} + +void registerCodecQATZSTD(CompressionCodecFactory & factory) +{ + factory.registerCompressionCodec("QATZSTD", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() > 1) + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "QATZSTD codec must have 1 parameter, given {}", arguments->children.size()); + + const auto children = arguments->children; + const auto * literal = children[0]->as(); + if (!literal) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "QATZSTD codec argument must be integer"); + + level = static_cast(literal->value.safeGet()); + if (level > CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL || level < CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, + "QATZSTD codec doesn't support level more than {} and lower than {} , given {}", + CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL, CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL, level); + } + + return std::make_shared(level); + }); +} + +CompressionCodecQATZSTD::CompressionCodecQATZSTD(int level_) + : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecQATZSTD")) +{ + setCodecDescription("QATZSTD", {std::make_shared(static_cast(level))}); +} + +CompressionCodecQATZSTD::~CompressionCodecQATZSTD() +{ + if (initialized) + { + /* Free sequence producer state */ + QZSTD_freeSeqProdState(sequenceProducerState); + ZSTD_freeCCtx(cctx); + } +} + +} +#endif diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 7eb7aaf3222..b7a790b2426 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -11,44 +10,10 @@ #include #include #include -#ifdef ENABLE_QATZSTD_COMPRESSION -#include "qatseqprod.h" -#endif +#include "CompressionCodecZSTD.h" namespace DB { - -class CompressionCodecZSTD : public ICompressionCodec -{ -public: - static constexpr auto ZSTD_DEFAULT_LEVEL = 1; - static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24; - - explicit CompressionCodecZSTD(int level_); - CompressionCodecZSTD(int level_, int window_log); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - -private: - const int level; - const bool enable_long_range; - const int window_log; -}; - - namespace ErrorCodes { extern const int CANNOT_COMPRESS; @@ -160,98 +125,6 @@ void registerCodecZSTD(CompressionCodecFactory & factory) }); } -#ifdef ENABLE_QATZSTD_COMPRESSION -class CompressionCodecQATZSTD : public CompressionCodecZSTD -{ -public: - static constexpr auto QATZSTD_SUPPORTED_MIN_LEVEL = 1; - static constexpr auto QATZSTD_SUPPORTED_MAX_LEVEL = 12; - explicit CompressionCodecQATZSTD(int level_); - ~CompressionCodecQATZSTD() override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - -private: - const int level; - mutable bool initialized; - mutable ZSTD_CCtx* cctx; - mutable void *sequenceProducerState; - Poco::Logger * log; -}; - -UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const -{ - if (!initialized) - { - cctx = ZSTD_createCCtx(); - /* Start QAT device, start QAT device at any time before compression job started */ - int res = QZSTD_startQatDevice(); - /* Create sequence producer state for QAT sequence producer */ - sequenceProducerState = QZSTD_createSeqProdState(); - /* register qatSequenceProducer */ - ZSTD_registerSequenceProducer( - cctx, - sequenceProducerState, - qatSequenceProducer - ); - /* Enable sequence producer fallback */ - ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); - initialized = true; - LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); - } - size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); - - if (ZSTD_isError(compressed_size)) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); - - return static_cast(compressed_size); -} - -void registerCodecQATZSTD(CompressionCodecFactory & factory) -{ - factory.registerCompressionCodec("QATZSTD", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr - { - int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; - if (arguments && !arguments->children.empty()) - { - if (arguments->children.size() > 1) - throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "QATZSTD codec must have 1 parameter, given {}", arguments->children.size()); - - const auto children = arguments->children; - const auto * literal = children[0]->as(); - if (!literal) - throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "QATZSTD codec argument must be integer"); - - level = static_cast(literal->value.safeGet()); - if (level > CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL || level < CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL) - throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, - "QATZSTD codec doesn't support level more than {} and lower than {} , given {}", - CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL, CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL, level); - } - - return std::make_shared(level); - }); -} - -CompressionCodecQATZSTD::CompressionCodecQATZSTD(int level_) - : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecQATZSTD")) -{ - setCodecDescription("QATZSTD", {std::make_shared(static_cast(level))}); -} - -CompressionCodecQATZSTD::~CompressionCodecQATZSTD() -{ - if (initialized) - { - /* Free sequence producer state */ - QZSTD_freeSeqProdState(sequenceProducerState); - ZSTD_freeCCtx(cctx); - } -} -#endif - CompressionCodecPtr getCompressionCodecZSTD(int level) { return std::make_shared(level); diff --git a/src/Compression/CompressionCodecZSTD.h b/src/Compression/CompressionCodecZSTD.h new file mode 100644 index 00000000000..e6799aa8d9f --- /dev/null +++ b/src/Compression/CompressionCodecZSTD.h @@ -0,0 +1,36 @@ +#include + +namespace DB +{ + +class CompressionCodecZSTD : public ICompressionCodec +{ +public: + static constexpr auto ZSTD_DEFAULT_LEVEL = 1; + static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24; + + explicit CompressionCodecZSTD(int level_); + CompressionCodecZSTD(int level_, int window_log); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + +private: + const int level; + const bool enable_long_range; + const int window_log; +}; + +} From 0d788b8dd1b3bfb5722a2fa15b2f25358a07ac03 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 11:27:24 -0500 Subject: [PATCH 019/116] refine defect --- contrib/qatlib-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 88d0cd4aa9e..183bfd7b386 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -47,7 +47,7 @@ target_include_directories(_osal PRIVATE ${OPENSSL_DIR}/include ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include) -target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL -D__GLIBC_PREREQ=0) +target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) #_qatlib SET(LIBQAT_Sources From 6154ea92bc60e51873ebcfbc4ed5ce9d5de87b30 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 12:51:03 -0500 Subject: [PATCH 020/116] add pragma once --- src/Compression/CompressionCodecZSTD.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Compression/CompressionCodecZSTD.h b/src/Compression/CompressionCodecZSTD.h index e6799aa8d9f..cdded9fc08a 100644 --- a/src/Compression/CompressionCodecZSTD.h +++ b/src/Compression/CompressionCodecZSTD.h @@ -1,3 +1,5 @@ +#pragma once + #include namespace DB From d2c4726a462c54fdc73b857ab39a13ac67215657 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 16:11:21 -0500 Subject: [PATCH 021/116] add a copied features.h --- contrib/qatlib-cmake/CMakeLists.txt | 1 - contrib/qatlib-cmake/include/features.h | 451 ++++++++++++++++++++++++ 2 files changed, 451 insertions(+), 1 deletion(-) create mode 100644 contrib/qatlib-cmake/include/features.h diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 183bfd7b386..3dc69ff4f25 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -40,7 +40,6 @@ SET(LIBOSAL_Sources ${LIBOSAL_DIR}/linux/user_space/OsalCryptoInterface.c) add_library(_osal ${LIBOSAL_Sources}) target_include_directories(_osal PRIVATE - ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/include ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include diff --git a/contrib/qatlib-cmake/include/features.h b/contrib/qatlib-cmake/include/features.h new file mode 100644 index 00000000000..9194ddf1d60 --- /dev/null +++ b/contrib/qatlib-cmake/include/features.h @@ -0,0 +1,451 @@ +/* Copyright (C) 1991-2018 Free Software Foundation, Inc. + This file is part of the GNU C Library. + + The GNU C Library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + The GNU C Library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with the GNU C Library; if not, see + . */ + +#ifndef _FEATURES_H +#define _FEATURES_H 1 + +/* These are defined by the user (or the compiler) + to specify the desired environment: + + __STRICT_ANSI__ ISO Standard C. + _ISOC99_SOURCE Extensions to ISO C89 from ISO C99. + _ISOC11_SOURCE Extensions to ISO C99 from ISO C11. + __STDC_WANT_LIB_EXT2__ + Extensions to ISO C99 from TR 27431-2:2010. + __STDC_WANT_IEC_60559_BFP_EXT__ + Extensions to ISO C11 from TS 18661-1:2014. + __STDC_WANT_IEC_60559_FUNCS_EXT__ + Extensions to ISO C11 from TS 18661-4:2015. + __STDC_WANT_IEC_60559_TYPES_EXT__ + Extensions to ISO C11 from TS 18661-3:2015. + + _POSIX_SOURCE IEEE Std 1003.1. + _POSIX_C_SOURCE If ==1, like _POSIX_SOURCE; if >=2 add IEEE Std 1003.2; + if >=199309L, add IEEE Std 1003.1b-1993; + if >=199506L, add IEEE Std 1003.1c-1995; + if >=200112L, all of IEEE 1003.1-2004 + if >=200809L, all of IEEE 1003.1-2008 + _XOPEN_SOURCE Includes POSIX and XPG things. Set to 500 if + Single Unix conformance is wanted, to 600 for the + sixth revision, to 700 for the seventh revision. + _XOPEN_SOURCE_EXTENDED XPG things and X/Open Unix extensions. + _LARGEFILE_SOURCE Some more functions for correct standard I/O. + _LARGEFILE64_SOURCE Additional functionality from LFS for large files. + _FILE_OFFSET_BITS=N Select default filesystem interface. + _ATFILE_SOURCE Additional *at interfaces. + _GNU_SOURCE All of the above, plus GNU extensions. + _DEFAULT_SOURCE The default set of features (taking precedence over + __STRICT_ANSI__). + + _FORTIFY_SOURCE Add security hardening to many library functions. + Set to 1 or 2; 2 performs stricter checks than 1. + + _REENTRANT, _THREAD_SAFE + Obsolete; equivalent to _POSIX_C_SOURCE=199506L. + + The `-ansi' switch to the GNU C compiler, and standards conformance + options such as `-std=c99', define __STRICT_ANSI__. If none of + these are defined, or if _DEFAULT_SOURCE is defined, the default is + to have _POSIX_SOURCE set to one and _POSIX_C_SOURCE set to + 200809L, as well as enabling miscellaneous functions from BSD and + SVID. If more than one of these are defined, they accumulate. For + example __STRICT_ANSI__, _POSIX_SOURCE and _POSIX_C_SOURCE together + give you ISO C, 1003.1, and 1003.2, but nothing else. + + These are defined by this file and are used by the + header files to decide what to declare or define: + + __GLIBC_USE (F) Define things from feature set F. This is defined + to 1 or 0; the subsequent macros are either defined + or undefined, and those tests should be moved to + __GLIBC_USE. + __USE_ISOC11 Define ISO C11 things. + __USE_ISOC99 Define ISO C99 things. + __USE_ISOC95 Define ISO C90 AMD1 (C95) things. + __USE_ISOCXX11 Define ISO C++11 things. + __USE_POSIX Define IEEE Std 1003.1 things. + __USE_POSIX2 Define IEEE Std 1003.2 things. + __USE_POSIX199309 Define IEEE Std 1003.1, and .1b things. + __USE_POSIX199506 Define IEEE Std 1003.1, .1b, .1c and .1i things. + __USE_XOPEN Define XPG things. + __USE_XOPEN_EXTENDED Define X/Open Unix things. + __USE_UNIX98 Define Single Unix V2 things. + __USE_XOPEN2K Define XPG6 things. + __USE_XOPEN2KXSI Define XPG6 XSI things. + __USE_XOPEN2K8 Define XPG7 things. + __USE_XOPEN2K8XSI Define XPG7 XSI things. + __USE_LARGEFILE Define correct standard I/O things. + __USE_LARGEFILE64 Define LFS things with separate names. + __USE_FILE_OFFSET64 Define 64bit interface as default. + __USE_MISC Define things from 4.3BSD or System V Unix. + __USE_ATFILE Define *at interfaces and AT_* constants for them. + __USE_GNU Define GNU extensions. + __USE_FORTIFY_LEVEL Additional security measures used, according to level. + + The macros `__GNU_LIBRARY__', `__GLIBC__', and `__GLIBC_MINOR__' are + defined by this file unconditionally. `__GNU_LIBRARY__' is provided + only for compatibility. All new code should use the other symbols + to test for features. + + All macros listed above as possibly being defined by this file are + explicitly undefined if they are not explicitly defined. + Feature-test macros that are not defined by the user or compiler + but are implied by the other feature-test macros defined (or by the + lack of any definitions) are defined by the file. + + ISO C feature test macros depend on the definition of the macro + when an affected header is included, not when the first system + header is included, and so they are handled in + , which does not have a multiple include + guard. Feature test macros that can be handled from the first + system header included are handled here. */ + + +/* Undefine everything, so we get a clean slate. */ +#undef __USE_ISOC11 +#undef __USE_ISOC99 +#undef __USE_ISOC95 +#undef __USE_ISOCXX11 +#undef __USE_POSIX +#undef __USE_POSIX2 +#undef __USE_POSIX199309 +#undef __USE_POSIX199506 +#undef __USE_XOPEN +#undef __USE_XOPEN_EXTENDED +#undef __USE_UNIX98 +#undef __USE_XOPEN2K +#undef __USE_XOPEN2KXSI +#undef __USE_XOPEN2K8 +#undef __USE_XOPEN2K8XSI +#undef __USE_LARGEFILE +#undef __USE_LARGEFILE64 +#undef __USE_FILE_OFFSET64 +#undef __USE_MISC +#undef __USE_ATFILE +#undef __USE_GNU +#undef __USE_FORTIFY_LEVEL +#undef __KERNEL_STRICT_NAMES +#undef __GLIBC_USE_DEPRECATED_GETS + +/* Suppress kernel-name space pollution unless user expressedly asks + for it. */ +#ifndef _LOOSE_KERNEL_NAMES +# define __KERNEL_STRICT_NAMES +#endif + +/* Convenience macro to test the version of gcc. + Use like this: + #if __GNUC_PREREQ (2,8) + ... code requiring gcc 2.8 or later ... + #endif + Note: only works for GCC 2.0 and later, because __GNUC_MINOR__ was + added in 2.0. */ +#if defined __GNUC__ && defined __GNUC_MINOR__ +# define __GNUC_PREREQ(maj, min) \ + ((__GNUC__ << 16) + __GNUC_MINOR__ >= ((maj) << 16) + (min)) +#else +# define __GNUC_PREREQ(maj, min) 0 +#endif + +/* Similarly for clang. Features added to GCC after version 4.2 may + or may not also be available in clang, and clang's definitions of + __GNUC(_MINOR)__ are fixed at 4 and 2 respectively. Not all such + features can be queried via __has_extension/__has_feature. */ +#if defined __clang_major__ && defined __clang_minor__ +# define __glibc_clang_prereq(maj, min) \ + ((__clang_major__ << 16) + __clang_minor__ >= ((maj) << 16) + (min)) +#else +# define __glibc_clang_prereq(maj, min) 0 +#endif + +/* Whether to use feature set F. */ +#define __GLIBC_USE(F) __GLIBC_USE_ ## F + +/* _BSD_SOURCE and _SVID_SOURCE are deprecated aliases for + _DEFAULT_SOURCE. If _DEFAULT_SOURCE is present we do not + issue a warning; the expectation is that the source is being + transitioned to use the new macro. */ +#if (defined _BSD_SOURCE || defined _SVID_SOURCE) \ + && !defined _DEFAULT_SOURCE +# warning "_BSD_SOURCE and _SVID_SOURCE are deprecated, use _DEFAULT_SOURCE" +# undef _DEFAULT_SOURCE +# define _DEFAULT_SOURCE 1 +#endif + +/* If _GNU_SOURCE was defined by the user, turn on all the other features. */ +#ifdef _GNU_SOURCE +# undef _ISOC95_SOURCE +# define _ISOC95_SOURCE 1 +# undef _ISOC99_SOURCE +# define _ISOC99_SOURCE 1 +# undef _ISOC11_SOURCE +# define _ISOC11_SOURCE 1 +# undef _POSIX_SOURCE +# define _POSIX_SOURCE 1 +# undef _POSIX_C_SOURCE +# define _POSIX_C_SOURCE 200809L +# undef _XOPEN_SOURCE +# define _XOPEN_SOURCE 700 +# undef _XOPEN_SOURCE_EXTENDED +# define _XOPEN_SOURCE_EXTENDED 1 +# undef _LARGEFILE64_SOURCE +# define _LARGEFILE64_SOURCE 1 +# undef _DEFAULT_SOURCE +# define _DEFAULT_SOURCE 1 +# undef _ATFILE_SOURCE +# define _ATFILE_SOURCE 1 +#endif + +/* If nothing (other than _GNU_SOURCE and _DEFAULT_SOURCE) is defined, + define _DEFAULT_SOURCE. */ +#if (defined _DEFAULT_SOURCE \ + || (!defined __STRICT_ANSI__ \ + && !defined _ISOC99_SOURCE \ + && !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE \ + && !defined _XOPEN_SOURCE)) +# undef _DEFAULT_SOURCE +# define _DEFAULT_SOURCE 1 +#endif + +/* This is to enable the ISO C11 extension. */ +#if (defined _ISOC11_SOURCE \ + || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 201112L)) +# define __USE_ISOC11 1 +#endif + +/* This is to enable the ISO C99 extension. */ +#if (defined _ISOC99_SOURCE || defined _ISOC11_SOURCE \ + || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 199901L)) +# define __USE_ISOC99 1 +#endif + +/* This is to enable the ISO C90 Amendment 1:1995 extension. */ +#if (defined _ISOC99_SOURCE || defined _ISOC11_SOURCE \ + || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 199409L)) +# define __USE_ISOC95 1 +#endif + +#ifdef __cplusplus +/* This is to enable compatibility for ISO C++17. */ +# if __cplusplus >= 201703L +# define __USE_ISOC11 1 +# endif +/* This is to enable compatibility for ISO C++11. + Check the temporary macro for now, too. */ +# if __cplusplus >= 201103L || defined __GXX_EXPERIMENTAL_CXX0X__ +# define __USE_ISOCXX11 1 +# define __USE_ISOC99 1 +# endif +#endif + +/* If none of the ANSI/POSIX macros are defined, or if _DEFAULT_SOURCE + is defined, use POSIX.1-2008 (or another version depending on + _XOPEN_SOURCE). */ +#ifdef _DEFAULT_SOURCE +# if !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE +# define __USE_POSIX_IMPLICITLY 1 +# endif +# undef _POSIX_SOURCE +# define _POSIX_SOURCE 1 +# undef _POSIX_C_SOURCE +# define _POSIX_C_SOURCE 200809L +#endif + +#if ((!defined __STRICT_ANSI__ \ + || (defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) >= 500)) \ + && !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE) +# define _POSIX_SOURCE 1 +# if defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 500 +# define _POSIX_C_SOURCE 2 +# elif defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 600 +# define _POSIX_C_SOURCE 199506L +# elif defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 700 +# define _POSIX_C_SOURCE 200112L +# else +# define _POSIX_C_SOURCE 200809L +# endif +# define __USE_POSIX_IMPLICITLY 1 +#endif + +/* Some C libraries once required _REENTRANT and/or _THREAD_SAFE to be + defined in all multithreaded code. GNU libc has not required this + for many years. We now treat them as compatibility synonyms for + _POSIX_C_SOURCE=199506L, which is the earliest level of POSIX with + comprehensive support for multithreaded code. Using them never + lowers the selected level of POSIX conformance, only raises it. */ +#if ((!defined _POSIX_C_SOURCE || (_POSIX_C_SOURCE - 0) < 199506L) \ + && (defined _REENTRANT || defined _THREAD_SAFE)) +# define _POSIX_SOURCE 1 +# undef _POSIX_C_SOURCE +# define _POSIX_C_SOURCE 199506L +#endif + +#if (defined _POSIX_SOURCE \ + || (defined _POSIX_C_SOURCE && _POSIX_C_SOURCE >= 1) \ + || defined _XOPEN_SOURCE) +# define __USE_POSIX 1 +#endif + +#if defined _POSIX_C_SOURCE && _POSIX_C_SOURCE >= 2 || defined _XOPEN_SOURCE +# define __USE_POSIX2 1 +#endif + +#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 199309L +# define __USE_POSIX199309 1 +#endif + +#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 199506L +# define __USE_POSIX199506 1 +#endif + +#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 200112L +# define __USE_XOPEN2K 1 +# undef __USE_ISOC95 +# define __USE_ISOC95 1 +# undef __USE_ISOC99 +# define __USE_ISOC99 1 +#endif + +#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 200809L +# define __USE_XOPEN2K8 1 +# undef _ATFILE_SOURCE +# define _ATFILE_SOURCE 1 +#endif + +#ifdef _XOPEN_SOURCE +# define __USE_XOPEN 1 +# if (_XOPEN_SOURCE - 0) >= 500 +# define __USE_XOPEN_EXTENDED 1 +# define __USE_UNIX98 1 +# undef _LARGEFILE_SOURCE +# define _LARGEFILE_SOURCE 1 +# if (_XOPEN_SOURCE - 0) >= 600 +# if (_XOPEN_SOURCE - 0) >= 700 +# define __USE_XOPEN2K8 1 +# define __USE_XOPEN2K8XSI 1 +# endif +# define __USE_XOPEN2K 1 +# define __USE_XOPEN2KXSI 1 +# undef __USE_ISOC95 +# define __USE_ISOC95 1 +# undef __USE_ISOC99 +# define __USE_ISOC99 1 +# endif +# else +# ifdef _XOPEN_SOURCE_EXTENDED +# define __USE_XOPEN_EXTENDED 1 +# endif +# endif +#endif + +#ifdef _LARGEFILE_SOURCE +# define __USE_LARGEFILE 1 +#endif + +#ifdef _LARGEFILE64_SOURCE +# define __USE_LARGEFILE64 1 +#endif + +#if defined _FILE_OFFSET_BITS && _FILE_OFFSET_BITS == 64 +# define __USE_FILE_OFFSET64 1 +#endif + +#if defined _DEFAULT_SOURCE +# define __USE_MISC 1 +#endif + +#ifdef _ATFILE_SOURCE +# define __USE_ATFILE 1 +#endif + +#ifdef _GNU_SOURCE +# define __USE_GNU 1 +#endif + +#if defined _FORTIFY_SOURCE && _FORTIFY_SOURCE > 0 \ + && __GNUC_PREREQ (4, 1) && defined __OPTIMIZE__ && __OPTIMIZE__ > 0 +# if _FORTIFY_SOURCE > 1 +# define __USE_FORTIFY_LEVEL 2 +# else +# define __USE_FORTIFY_LEVEL 1 +# endif +#else +# define __USE_FORTIFY_LEVEL 0 +#endif + +/* The function 'gets' existed in C89, but is impossible to use + safely. It has been removed from ISO C11 and ISO C++14. Note: for + compatibility with various implementations of , this test + must consider only the value of __cplusplus when compiling C++. */ +#if defined __cplusplus ? __cplusplus >= 201402L : defined __USE_ISOC11 +# define __GLIBC_USE_DEPRECATED_GETS 0 +#else +# define __GLIBC_USE_DEPRECATED_GETS 1 +#endif + +/* Get definitions of __STDC_* predefined macros, if the compiler has + not preincluded this header automatically. */ +#include + +/* This macro indicates that the installed library is the GNU C Library. + For historic reasons the value now is 6 and this will stay from now + on. The use of this variable is deprecated. Use __GLIBC__ and + __GLIBC_MINOR__ now (see below) when you want to test for a specific + GNU C library version and use the values in to get + the sonames of the shared libraries. */ +#undef __GNU_LIBRARY__ +#define __GNU_LIBRARY__ 6 + +/* Major and minor version number of the GNU C library package. Use + these macros to test for features in specific releases. */ +#define __GLIBC__ 2 +#define __GLIBC_MINOR__ 27 + +#define __GLIBC_PREREQ(maj, min) \ + ((__GLIBC__ << 16) + __GLIBC_MINOR__ >= ((maj) << 16) + (min)) + +/* This is here only because every header file already includes this one. */ +#ifndef __ASSEMBLER__ +# ifndef _SYS_CDEFS_H +# include +# endif + +/* If we don't have __REDIRECT, prototypes will be missing if + __USE_FILE_OFFSET64 but not __USE_LARGEFILE[64]. */ +# if defined __USE_FILE_OFFSET64 && !defined __REDIRECT +# define __USE_LARGEFILE 1 +# define __USE_LARGEFILE64 1 +# endif + +#endif /* !ASSEMBLER */ + +/* Decide whether we can define 'extern inline' functions in headers. */ +#if __GNUC_PREREQ (2, 7) && defined __OPTIMIZE__ \ + && !defined __OPTIMIZE_SIZE__ && !defined __NO_INLINE__ \ + && defined __extern_inline +# define __USE_EXTERN_INLINES 1 +#endif + + +/* This is here only because every header file already includes this one. + Get the definitions of all the appropriate `__stub_FUNCTION' symbols. + contains `#define __stub_FUNCTION' when FUNCTION is a stub + that will always return failure (and set errno to ENOSYS). */ +#include + + +#endif /* features.h */ From 846cbc1abe5b28af7340d680d8cdfbe035548a8f Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Dec 2023 18:22:56 -0500 Subject: [PATCH 022/116] fixed header issue --- contrib/qatlib-cmake/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 3dc69ff4f25..57c1b3a4cb3 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -45,7 +45,8 @@ target_include_directories(_osal PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include ${OPENSSL_DIR}/include ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include - ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include) + ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include + ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/include) target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) #_qatlib From b4a945ec89d90313b8698a78152b9a128b1fe887 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 13 Dec 2023 10:41:46 -0500 Subject: [PATCH 023/116] update mqueue.h for build conflict --- contrib/qatlib-cmake/CMakeLists.txt | 3 +- contrib/qatlib-cmake/include/features.h | 451 ------------------------ contrib/qatlib-cmake/include/mqueue.h | 14 + 3 files changed, 15 insertions(+), 453 deletions(-) delete mode 100644 contrib/qatlib-cmake/include/features.h diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 57c1b3a4cb3..3dc69ff4f25 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -45,8 +45,7 @@ target_include_directories(_osal PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include ${OPENSSL_DIR}/include ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include - ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include - ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/include) + ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include) target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) #_qatlib diff --git a/contrib/qatlib-cmake/include/features.h b/contrib/qatlib-cmake/include/features.h deleted file mode 100644 index 9194ddf1d60..00000000000 --- a/contrib/qatlib-cmake/include/features.h +++ /dev/null @@ -1,451 +0,0 @@ -/* Copyright (C) 1991-2018 Free Software Foundation, Inc. - This file is part of the GNU C Library. - - The GNU C Library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - The GNU C Library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with the GNU C Library; if not, see - . */ - -#ifndef _FEATURES_H -#define _FEATURES_H 1 - -/* These are defined by the user (or the compiler) - to specify the desired environment: - - __STRICT_ANSI__ ISO Standard C. - _ISOC99_SOURCE Extensions to ISO C89 from ISO C99. - _ISOC11_SOURCE Extensions to ISO C99 from ISO C11. - __STDC_WANT_LIB_EXT2__ - Extensions to ISO C99 from TR 27431-2:2010. - __STDC_WANT_IEC_60559_BFP_EXT__ - Extensions to ISO C11 from TS 18661-1:2014. - __STDC_WANT_IEC_60559_FUNCS_EXT__ - Extensions to ISO C11 from TS 18661-4:2015. - __STDC_WANT_IEC_60559_TYPES_EXT__ - Extensions to ISO C11 from TS 18661-3:2015. - - _POSIX_SOURCE IEEE Std 1003.1. - _POSIX_C_SOURCE If ==1, like _POSIX_SOURCE; if >=2 add IEEE Std 1003.2; - if >=199309L, add IEEE Std 1003.1b-1993; - if >=199506L, add IEEE Std 1003.1c-1995; - if >=200112L, all of IEEE 1003.1-2004 - if >=200809L, all of IEEE 1003.1-2008 - _XOPEN_SOURCE Includes POSIX and XPG things. Set to 500 if - Single Unix conformance is wanted, to 600 for the - sixth revision, to 700 for the seventh revision. - _XOPEN_SOURCE_EXTENDED XPG things and X/Open Unix extensions. - _LARGEFILE_SOURCE Some more functions for correct standard I/O. - _LARGEFILE64_SOURCE Additional functionality from LFS for large files. - _FILE_OFFSET_BITS=N Select default filesystem interface. - _ATFILE_SOURCE Additional *at interfaces. - _GNU_SOURCE All of the above, plus GNU extensions. - _DEFAULT_SOURCE The default set of features (taking precedence over - __STRICT_ANSI__). - - _FORTIFY_SOURCE Add security hardening to many library functions. - Set to 1 or 2; 2 performs stricter checks than 1. - - _REENTRANT, _THREAD_SAFE - Obsolete; equivalent to _POSIX_C_SOURCE=199506L. - - The `-ansi' switch to the GNU C compiler, and standards conformance - options such as `-std=c99', define __STRICT_ANSI__. If none of - these are defined, or if _DEFAULT_SOURCE is defined, the default is - to have _POSIX_SOURCE set to one and _POSIX_C_SOURCE set to - 200809L, as well as enabling miscellaneous functions from BSD and - SVID. If more than one of these are defined, they accumulate. For - example __STRICT_ANSI__, _POSIX_SOURCE and _POSIX_C_SOURCE together - give you ISO C, 1003.1, and 1003.2, but nothing else. - - These are defined by this file and are used by the - header files to decide what to declare or define: - - __GLIBC_USE (F) Define things from feature set F. This is defined - to 1 or 0; the subsequent macros are either defined - or undefined, and those tests should be moved to - __GLIBC_USE. - __USE_ISOC11 Define ISO C11 things. - __USE_ISOC99 Define ISO C99 things. - __USE_ISOC95 Define ISO C90 AMD1 (C95) things. - __USE_ISOCXX11 Define ISO C++11 things. - __USE_POSIX Define IEEE Std 1003.1 things. - __USE_POSIX2 Define IEEE Std 1003.2 things. - __USE_POSIX199309 Define IEEE Std 1003.1, and .1b things. - __USE_POSIX199506 Define IEEE Std 1003.1, .1b, .1c and .1i things. - __USE_XOPEN Define XPG things. - __USE_XOPEN_EXTENDED Define X/Open Unix things. - __USE_UNIX98 Define Single Unix V2 things. - __USE_XOPEN2K Define XPG6 things. - __USE_XOPEN2KXSI Define XPG6 XSI things. - __USE_XOPEN2K8 Define XPG7 things. - __USE_XOPEN2K8XSI Define XPG7 XSI things. - __USE_LARGEFILE Define correct standard I/O things. - __USE_LARGEFILE64 Define LFS things with separate names. - __USE_FILE_OFFSET64 Define 64bit interface as default. - __USE_MISC Define things from 4.3BSD or System V Unix. - __USE_ATFILE Define *at interfaces and AT_* constants for them. - __USE_GNU Define GNU extensions. - __USE_FORTIFY_LEVEL Additional security measures used, according to level. - - The macros `__GNU_LIBRARY__', `__GLIBC__', and `__GLIBC_MINOR__' are - defined by this file unconditionally. `__GNU_LIBRARY__' is provided - only for compatibility. All new code should use the other symbols - to test for features. - - All macros listed above as possibly being defined by this file are - explicitly undefined if they are not explicitly defined. - Feature-test macros that are not defined by the user or compiler - but are implied by the other feature-test macros defined (or by the - lack of any definitions) are defined by the file. - - ISO C feature test macros depend on the definition of the macro - when an affected header is included, not when the first system - header is included, and so they are handled in - , which does not have a multiple include - guard. Feature test macros that can be handled from the first - system header included are handled here. */ - - -/* Undefine everything, so we get a clean slate. */ -#undef __USE_ISOC11 -#undef __USE_ISOC99 -#undef __USE_ISOC95 -#undef __USE_ISOCXX11 -#undef __USE_POSIX -#undef __USE_POSIX2 -#undef __USE_POSIX199309 -#undef __USE_POSIX199506 -#undef __USE_XOPEN -#undef __USE_XOPEN_EXTENDED -#undef __USE_UNIX98 -#undef __USE_XOPEN2K -#undef __USE_XOPEN2KXSI -#undef __USE_XOPEN2K8 -#undef __USE_XOPEN2K8XSI -#undef __USE_LARGEFILE -#undef __USE_LARGEFILE64 -#undef __USE_FILE_OFFSET64 -#undef __USE_MISC -#undef __USE_ATFILE -#undef __USE_GNU -#undef __USE_FORTIFY_LEVEL -#undef __KERNEL_STRICT_NAMES -#undef __GLIBC_USE_DEPRECATED_GETS - -/* Suppress kernel-name space pollution unless user expressedly asks - for it. */ -#ifndef _LOOSE_KERNEL_NAMES -# define __KERNEL_STRICT_NAMES -#endif - -/* Convenience macro to test the version of gcc. - Use like this: - #if __GNUC_PREREQ (2,8) - ... code requiring gcc 2.8 or later ... - #endif - Note: only works for GCC 2.0 and later, because __GNUC_MINOR__ was - added in 2.0. */ -#if defined __GNUC__ && defined __GNUC_MINOR__ -# define __GNUC_PREREQ(maj, min) \ - ((__GNUC__ << 16) + __GNUC_MINOR__ >= ((maj) << 16) + (min)) -#else -# define __GNUC_PREREQ(maj, min) 0 -#endif - -/* Similarly for clang. Features added to GCC after version 4.2 may - or may not also be available in clang, and clang's definitions of - __GNUC(_MINOR)__ are fixed at 4 and 2 respectively. Not all such - features can be queried via __has_extension/__has_feature. */ -#if defined __clang_major__ && defined __clang_minor__ -# define __glibc_clang_prereq(maj, min) \ - ((__clang_major__ << 16) + __clang_minor__ >= ((maj) << 16) + (min)) -#else -# define __glibc_clang_prereq(maj, min) 0 -#endif - -/* Whether to use feature set F. */ -#define __GLIBC_USE(F) __GLIBC_USE_ ## F - -/* _BSD_SOURCE and _SVID_SOURCE are deprecated aliases for - _DEFAULT_SOURCE. If _DEFAULT_SOURCE is present we do not - issue a warning; the expectation is that the source is being - transitioned to use the new macro. */ -#if (defined _BSD_SOURCE || defined _SVID_SOURCE) \ - && !defined _DEFAULT_SOURCE -# warning "_BSD_SOURCE and _SVID_SOURCE are deprecated, use _DEFAULT_SOURCE" -# undef _DEFAULT_SOURCE -# define _DEFAULT_SOURCE 1 -#endif - -/* If _GNU_SOURCE was defined by the user, turn on all the other features. */ -#ifdef _GNU_SOURCE -# undef _ISOC95_SOURCE -# define _ISOC95_SOURCE 1 -# undef _ISOC99_SOURCE -# define _ISOC99_SOURCE 1 -# undef _ISOC11_SOURCE -# define _ISOC11_SOURCE 1 -# undef _POSIX_SOURCE -# define _POSIX_SOURCE 1 -# undef _POSIX_C_SOURCE -# define _POSIX_C_SOURCE 200809L -# undef _XOPEN_SOURCE -# define _XOPEN_SOURCE 700 -# undef _XOPEN_SOURCE_EXTENDED -# define _XOPEN_SOURCE_EXTENDED 1 -# undef _LARGEFILE64_SOURCE -# define _LARGEFILE64_SOURCE 1 -# undef _DEFAULT_SOURCE -# define _DEFAULT_SOURCE 1 -# undef _ATFILE_SOURCE -# define _ATFILE_SOURCE 1 -#endif - -/* If nothing (other than _GNU_SOURCE and _DEFAULT_SOURCE) is defined, - define _DEFAULT_SOURCE. */ -#if (defined _DEFAULT_SOURCE \ - || (!defined __STRICT_ANSI__ \ - && !defined _ISOC99_SOURCE \ - && !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE \ - && !defined _XOPEN_SOURCE)) -# undef _DEFAULT_SOURCE -# define _DEFAULT_SOURCE 1 -#endif - -/* This is to enable the ISO C11 extension. */ -#if (defined _ISOC11_SOURCE \ - || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 201112L)) -# define __USE_ISOC11 1 -#endif - -/* This is to enable the ISO C99 extension. */ -#if (defined _ISOC99_SOURCE || defined _ISOC11_SOURCE \ - || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 199901L)) -# define __USE_ISOC99 1 -#endif - -/* This is to enable the ISO C90 Amendment 1:1995 extension. */ -#if (defined _ISOC99_SOURCE || defined _ISOC11_SOURCE \ - || (defined __STDC_VERSION__ && __STDC_VERSION__ >= 199409L)) -# define __USE_ISOC95 1 -#endif - -#ifdef __cplusplus -/* This is to enable compatibility for ISO C++17. */ -# if __cplusplus >= 201703L -# define __USE_ISOC11 1 -# endif -/* This is to enable compatibility for ISO C++11. - Check the temporary macro for now, too. */ -# if __cplusplus >= 201103L || defined __GXX_EXPERIMENTAL_CXX0X__ -# define __USE_ISOCXX11 1 -# define __USE_ISOC99 1 -# endif -#endif - -/* If none of the ANSI/POSIX macros are defined, or if _DEFAULT_SOURCE - is defined, use POSIX.1-2008 (or another version depending on - _XOPEN_SOURCE). */ -#ifdef _DEFAULT_SOURCE -# if !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE -# define __USE_POSIX_IMPLICITLY 1 -# endif -# undef _POSIX_SOURCE -# define _POSIX_SOURCE 1 -# undef _POSIX_C_SOURCE -# define _POSIX_C_SOURCE 200809L -#endif - -#if ((!defined __STRICT_ANSI__ \ - || (defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) >= 500)) \ - && !defined _POSIX_SOURCE && !defined _POSIX_C_SOURCE) -# define _POSIX_SOURCE 1 -# if defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 500 -# define _POSIX_C_SOURCE 2 -# elif defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 600 -# define _POSIX_C_SOURCE 199506L -# elif defined _XOPEN_SOURCE && (_XOPEN_SOURCE - 0) < 700 -# define _POSIX_C_SOURCE 200112L -# else -# define _POSIX_C_SOURCE 200809L -# endif -# define __USE_POSIX_IMPLICITLY 1 -#endif - -/* Some C libraries once required _REENTRANT and/or _THREAD_SAFE to be - defined in all multithreaded code. GNU libc has not required this - for many years. We now treat them as compatibility synonyms for - _POSIX_C_SOURCE=199506L, which is the earliest level of POSIX with - comprehensive support for multithreaded code. Using them never - lowers the selected level of POSIX conformance, only raises it. */ -#if ((!defined _POSIX_C_SOURCE || (_POSIX_C_SOURCE - 0) < 199506L) \ - && (defined _REENTRANT || defined _THREAD_SAFE)) -# define _POSIX_SOURCE 1 -# undef _POSIX_C_SOURCE -# define _POSIX_C_SOURCE 199506L -#endif - -#if (defined _POSIX_SOURCE \ - || (defined _POSIX_C_SOURCE && _POSIX_C_SOURCE >= 1) \ - || defined _XOPEN_SOURCE) -# define __USE_POSIX 1 -#endif - -#if defined _POSIX_C_SOURCE && _POSIX_C_SOURCE >= 2 || defined _XOPEN_SOURCE -# define __USE_POSIX2 1 -#endif - -#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 199309L -# define __USE_POSIX199309 1 -#endif - -#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 199506L -# define __USE_POSIX199506 1 -#endif - -#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 200112L -# define __USE_XOPEN2K 1 -# undef __USE_ISOC95 -# define __USE_ISOC95 1 -# undef __USE_ISOC99 -# define __USE_ISOC99 1 -#endif - -#if defined _POSIX_C_SOURCE && (_POSIX_C_SOURCE - 0) >= 200809L -# define __USE_XOPEN2K8 1 -# undef _ATFILE_SOURCE -# define _ATFILE_SOURCE 1 -#endif - -#ifdef _XOPEN_SOURCE -# define __USE_XOPEN 1 -# if (_XOPEN_SOURCE - 0) >= 500 -# define __USE_XOPEN_EXTENDED 1 -# define __USE_UNIX98 1 -# undef _LARGEFILE_SOURCE -# define _LARGEFILE_SOURCE 1 -# if (_XOPEN_SOURCE - 0) >= 600 -# if (_XOPEN_SOURCE - 0) >= 700 -# define __USE_XOPEN2K8 1 -# define __USE_XOPEN2K8XSI 1 -# endif -# define __USE_XOPEN2K 1 -# define __USE_XOPEN2KXSI 1 -# undef __USE_ISOC95 -# define __USE_ISOC95 1 -# undef __USE_ISOC99 -# define __USE_ISOC99 1 -# endif -# else -# ifdef _XOPEN_SOURCE_EXTENDED -# define __USE_XOPEN_EXTENDED 1 -# endif -# endif -#endif - -#ifdef _LARGEFILE_SOURCE -# define __USE_LARGEFILE 1 -#endif - -#ifdef _LARGEFILE64_SOURCE -# define __USE_LARGEFILE64 1 -#endif - -#if defined _FILE_OFFSET_BITS && _FILE_OFFSET_BITS == 64 -# define __USE_FILE_OFFSET64 1 -#endif - -#if defined _DEFAULT_SOURCE -# define __USE_MISC 1 -#endif - -#ifdef _ATFILE_SOURCE -# define __USE_ATFILE 1 -#endif - -#ifdef _GNU_SOURCE -# define __USE_GNU 1 -#endif - -#if defined _FORTIFY_SOURCE && _FORTIFY_SOURCE > 0 \ - && __GNUC_PREREQ (4, 1) && defined __OPTIMIZE__ && __OPTIMIZE__ > 0 -# if _FORTIFY_SOURCE > 1 -# define __USE_FORTIFY_LEVEL 2 -# else -# define __USE_FORTIFY_LEVEL 1 -# endif -#else -# define __USE_FORTIFY_LEVEL 0 -#endif - -/* The function 'gets' existed in C89, but is impossible to use - safely. It has been removed from ISO C11 and ISO C++14. Note: for - compatibility with various implementations of , this test - must consider only the value of __cplusplus when compiling C++. */ -#if defined __cplusplus ? __cplusplus >= 201402L : defined __USE_ISOC11 -# define __GLIBC_USE_DEPRECATED_GETS 0 -#else -# define __GLIBC_USE_DEPRECATED_GETS 1 -#endif - -/* Get definitions of __STDC_* predefined macros, if the compiler has - not preincluded this header automatically. */ -#include - -/* This macro indicates that the installed library is the GNU C Library. - For historic reasons the value now is 6 and this will stay from now - on. The use of this variable is deprecated. Use __GLIBC__ and - __GLIBC_MINOR__ now (see below) when you want to test for a specific - GNU C library version and use the values in to get - the sonames of the shared libraries. */ -#undef __GNU_LIBRARY__ -#define __GNU_LIBRARY__ 6 - -/* Major and minor version number of the GNU C library package. Use - these macros to test for features in specific releases. */ -#define __GLIBC__ 2 -#define __GLIBC_MINOR__ 27 - -#define __GLIBC_PREREQ(maj, min) \ - ((__GLIBC__ << 16) + __GLIBC_MINOR__ >= ((maj) << 16) + (min)) - -/* This is here only because every header file already includes this one. */ -#ifndef __ASSEMBLER__ -# ifndef _SYS_CDEFS_H -# include -# endif - -/* If we don't have __REDIRECT, prototypes will be missing if - __USE_FILE_OFFSET64 but not __USE_LARGEFILE[64]. */ -# if defined __USE_FILE_OFFSET64 && !defined __REDIRECT -# define __USE_LARGEFILE 1 -# define __USE_LARGEFILE64 1 -# endif - -#endif /* !ASSEMBLER */ - -/* Decide whether we can define 'extern inline' functions in headers. */ -#if __GNUC_PREREQ (2, 7) && defined __OPTIMIZE__ \ - && !defined __OPTIMIZE_SIZE__ && !defined __NO_INLINE__ \ - && defined __extern_inline -# define __USE_EXTERN_INLINES 1 -#endif - - -/* This is here only because every header file already includes this one. - Get the definitions of all the appropriate `__stub_FUNCTION' symbols. - contains `#define __stub_FUNCTION' when FUNCTION is a stub - that will always return failure (and set errno to ENOSYS). */ -#include - - -#endif /* features.h */ diff --git a/contrib/qatlib-cmake/include/mqueue.h b/contrib/qatlib-cmake/include/mqueue.h index e69de29bb2d..7b1125074a8 100644 --- a/contrib/qatlib-cmake/include/mqueue.h +++ b/contrib/qatlib-cmake/include/mqueue.h @@ -0,0 +1,14 @@ +/* This is a workaround for a build conflict issue +1. __GLIBC_PREREQ (referenced in OsalServices.c) is only defined in './sysroot/linux-x86_64/include/features.h' +2. mqueue.h only exist under './sysroot/linux-x86_64-musl/' +This cause target_include_directories for _osal has a conflict between './sysroot/linux-x86_64/include' and './sysroot/linux-x86_64-musl/' +hence create mqueue.h separately under ./qatlib-cmake/include as an alternative. +*/ + +/* Major and minor version number of the GNU C library package. Use + these macros to test for features in specific releases. */ +#define __GLIBC__ 2 +#define __GLIBC_MINOR__ 27 + +#define __GLIBC_PREREQ(maj, min) \ + ((__GLIBC__ << 16) + __GLIBC_MINOR__ >= ((maj) << 16) + (min)) From 5e1b048e529d5f1c680a51b7123efc1b01ce5fe2 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 13 Dec 2023 11:02:06 -0500 Subject: [PATCH 024/116] refine ZSTDQAT code name --- contrib/CMakeLists.txt | 20 ++++----- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 6 +-- ...ATZSTD.cpp => CompressionCodecZSTDQAT.cpp} | 42 +++++++++---------- src/Compression/CompressionFactory.cpp | 12 +++--- 4 files changed, 40 insertions(+), 40 deletions(-) rename src/Compression/{CompressionCodecQATZSTD.cpp => CompressionCodecZSTDQAT.cpp} (70%) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 05f308508d4..387bc84f126 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -188,17 +188,17 @@ elseif(ENABLE_QATLIB) message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64 arch") endif() if (ENABLE_QATLIB) - option (ENABLE_USDM_DRV "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF) - option (ENABLE_QAT_OUTOFTREE "Using out-of-tree driver, user needs customize ICP_ROOT variable" OFF) + option (ENABLE_QAT_USDM_DRIVER "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF) + option (ENABLE_QAT_OUT_OF_TREE_BUILD "Using out-of-tree driver, user needs customize ICP_ROOT variable" OFF) set(ICP_ROOT "" CACHE STRING "ICP_ROOT variable to define the path of out-of-tree driver package") -if (ENABLE_QAT_OUTOFTREE) -if (ICP_ROOT STREQUAL "") - message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUTOFTREE=OFF; \ - If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") -endif () -else() - add_contrib (qatlib-cmake qatlib) -endif () + if (ENABLE_QAT_OUT_OF_TREE_BUILD) + if (ICP_ROOT STREQUAL "") + message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUT_OF_TREE_BUILD=OFF; \ + If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") + endif () + else() + add_contrib (qatlib-cmake qatlib) # requires: isa-l + endif () add_contrib (QAT-ZSTD-Plugin-cmake QAT-ZSTD-Plugin) else() message(STATUS "Not using QAT") diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index edc41866edd..43e085104a8 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -3,7 +3,7 @@ # meanwhile, user need to set ICP_ROOT environment variable which point to the root directory of QAT driver source tree. # in-tree means kernel has built-in qat driver, QAT-ZSTD-PLUGIN just has dependency on qatlib. -if (ENABLE_QAT_OUTOFTREE) +if (ENABLE_QAT_OUT_OF_TREE_BUILD) # Intel QATZSTD out-of-tree build message(STATUS "Intel QATZSTD out-of-tree build, ICP_ROOT:${ICP_ROOT}") @@ -16,7 +16,7 @@ set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") -if (ENABLE_USDM_DRV) +if (ENABLE_QAT_USDM_DRIVER) add_definitions(-DENABLE_USDM_DRV) endif() add_library(_qatzstd_plugin ${QATZSTD_SRC}) @@ -69,7 +69,7 @@ file(COPY "${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv/qae_mem.h" DESTINATION "${LIBQAT_HEADER_DIR}/qat/" ) -if (ENABLE_USDM_DRV) +if (ENABLE_QAT_USDM_DRIVER) add_definitions(-DENABLE_USDM_DRV) endif() diff --git a/src/Compression/CompressionCodecQATZSTD.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp similarity index 70% rename from src/Compression/CompressionCodecQATZSTD.cpp rename to src/Compression/CompressionCodecZSTDQAT.cpp index eca1adc6f25..376726b6195 100644 --- a/src/Compression/CompressionCodecQATZSTD.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,4 +1,5 @@ -#ifdef ENABLE_QATZSTD_COMPRESSION +#ifdef ENABLE_ZSTDQAT_COMPRESSION +#include #include #include #include @@ -8,8 +9,7 @@ #include #include #include -#include "qatseqprod.h" -#include "CompressionCodecZSTD.h" +#include namespace DB { @@ -20,13 +20,13 @@ namespace ErrorCodes extern const int ILLEGAL_CODEC_PARAMETER; } -class CompressionCodecQATZSTD : public CompressionCodecZSTD +class CompressionCodecZSTDQAT : public CompressionCodecZSTD { public: - static constexpr auto QATZSTD_SUPPORTED_MIN_LEVEL = 1; - static constexpr auto QATZSTD_SUPPORTED_MAX_LEVEL = 12; - explicit CompressionCodecQATZSTD(int level_); - ~CompressionCodecQATZSTD() override; + static constexpr auto ZSTDQAT_SUPPORTED_MIN_LEVEL = 1; + static constexpr auto ZSTDQAT_SUPPORTED_MAX_LEVEL = 12; + explicit CompressionCodecZSTDQAT(int level_); + ~CompressionCodecZSTDQAT() override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; @@ -39,7 +39,7 @@ private: Poco::Logger * log; }; -UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const +UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 source_size, char * dest) const { if (!initialized) { @@ -68,39 +68,39 @@ UInt32 CompressionCodecQATZSTD::doCompressData(const char * source, UInt32 sourc return static_cast(compressed_size); } -void registerCodecQATZSTD(CompressionCodecFactory & factory) +void registerCodecZSTDQAT(CompressionCodecFactory & factory) { - factory.registerCompressionCodec("QATZSTD", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + factory.registerCompressionCodec("ZSTDQAT", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr { int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; if (arguments && !arguments->children.empty()) { if (arguments->children.size() > 1) - throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "QATZSTD codec must have 1 parameter, given {}", arguments->children.size()); + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "ZSTDQAT codec must have 1 parameter, given {}", arguments->children.size()); const auto children = arguments->children; const auto * literal = children[0]->as(); if (!literal) - throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "QATZSTD codec argument must be integer"); + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTDQAT codec argument must be integer"); level = static_cast(literal->value.safeGet()); - if (level > CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL || level < CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL) + if (level > CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL || level < CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL) throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, - "QATZSTD codec doesn't support level more than {} and lower than {} , given {}", - CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MAX_LEVEL, CompressionCodecQATZSTD::QATZSTD_SUPPORTED_MIN_LEVEL, level); + "ZSTDQAT codec doesn't support level more than {} and lower than {} , given {}", + CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL, CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL, level); } - return std::make_shared(level); + return std::make_shared(level); }); } -CompressionCodecQATZSTD::CompressionCodecQATZSTD(int level_) - : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecQATZSTD")) +CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) + : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecZSTDQAT")) { - setCodecDescription("QATZSTD", {std::make_shared(static_cast(level))}); + setCodecDescription("ZSTDQAT", {std::make_shared(static_cast(level))}); } -CompressionCodecQATZSTD::~CompressionCodecQATZSTD() +CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() { if (initialized) { diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 96c245bb021..6cc34c6aa91 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -167,13 +167,13 @@ void registerCodecNone(CompressionCodecFactory & factory); void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); +#ifdef ENABLE_ZSTDQAT_COMPRESSION +void registerCodecZSTDQAT(CompressionCodecFactory & factory); +#endif void registerCodecMultiple(CompressionCodecFactory & factory); #ifdef ENABLE_QPL_COMPRESSION void registerCodecDeflateQpl(CompressionCodecFactory & factory); #endif -#ifdef ENABLE_QATZSTD_COMPRESSION -void registerCodecQATZSTD(CompressionCodecFactory & factory); -#endif /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build @@ -192,6 +192,9 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecNone(*this); registerCodecLZ4(*this); registerCodecZSTD(*this); +#ifdef ENABLE_ZSTDQAT_COMPRESSION + registerCodecZSTDQAT(*this); +#endif registerCodecLZ4HC(*this); registerCodecMultiple(*this); #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD @@ -203,9 +206,6 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecFPC(*this); #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflateQpl(*this); -#endif -#ifdef ENABLE_QATZSTD_COMPRESSION - registerCodecQATZSTD(*this); #endif registerCodecGCD(*this); #endif From bce1814363daef14b5e8f3eeaf7736dcfe1f6341 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 13 Dec 2023 11:18:21 -0500 Subject: [PATCH 025/116] add codec into doc --- docs/en/sql-reference/statements/create/table.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 602feb69d8a..d1c23c85c26 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -376,6 +376,12 @@ ClickHouse supports general purpose codecs and specialized codecs. High compression levels are useful for asymmetric scenarios, like compress once, decompress repeatedly. Higher levels mean better compression and higher CPU usage. +#### ZSTDQAT + +`ZSTDQAT[(level)]` — ZSTD QAT (hardware-accelerated compression) implemented by [QAT-ZSTD-Plugin](https://github.com/intel/QAT-ZSTD-Plugin) with configurable level. Default level: 1. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[6, 12\]. + +ZSTDQAT works best if the system has Intel® QAT ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)) offloading device. If no device available, it will fallback to native ZSTD compression. + #### DEFLATE_QPL `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: From 606a49444747fc7762b8e241d0a79e991d97267b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 13 Dec 2023 12:53:53 -0500 Subject: [PATCH 026/116] add dictionary --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index cbfa70eff32..936bb4aa5d5 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2662,3 +2662,5 @@ znode znodes zookeeperSessionUptime zstd +QuickAssist +ZSTDQAT From 14b8d8fb9291b2fe3fe3d2e65316c2ed8074f813 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 11 Dec 2023 19:43:56 +0800 Subject: [PATCH 027/116] imporve multiif --- src/Functions/multiIf.cpp | 56 ++++++++++++++++++- .../0_stateless/02918_multif_for_nullable.sh | 13 +++++ 2 files changed, 66 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02918_multif_for_nullable.sh diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index cdb9ca061c3..8363768c0fb 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -238,7 +238,7 @@ public: } const auto & settings = context->getSettingsRef(); - const WhichDataType which(result_type); + const WhichDataType which(removeNullable(result_type)); bool execute_multiif_columnar = settings.allow_execute_multiif_columnar && !contains_short && (which.isInt() || which.isUInt() || which.isFloat()); @@ -254,8 +254,17 @@ public: if (which.is##TYPE()) \ { \ MutableColumnPtr res = ColumnVector::create(rows); \ - executeInstructionsColumnar(instructions, rows, res); \ - return std::move(res); \ + if (!result_type->isNullable()) \ + { \ + executeInstructionsColumnar(instructions, rows, res); \ + return std::move(res); \ + } \ + else \ + { \ + MutableColumnPtr null_map = ColumnUInt8::create(rows); \ + executeInstructionsColumnarForNullable(instructions, rows, res, null_map); \ + return ColumnNullable::create(std::move(res), std::move(null_map)); \ + } \ } #define ENUMERATE_NUMERIC_TYPES(M, INDEX) \ @@ -373,6 +382,47 @@ private: } } + template + static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map) + { + PaddedPODArray inserts(rows, static_cast(instructions.size())); + calculateInserts(instructions, rows, inserts); + + PaddedPODArray & res_data = assert_cast &>(*res).getData(); + PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); + std::vector *> data_cols(instructions.size()); + std::vector null_map_cols(instructions.size()); + for (size_t i = 0; i < instructions.size(); ++i) + { + if (instructions[i].source->isNullable()) + { + const ColumnNullable * nullable_col; + if (!instructions[i].source_is_constant) + nullable_col = assert_cast(instructions[i].source.get()); + else + { + const ColumnPtr data_column = assert_cast(instructions[i].source.get())->getDataColumnPtr(); + nullable_col = assert_cast(data_column.get()); + } + null_map_cols[i] = assert_cast(nullable_col->getNullMapColumnPtr().get()); + data_cols[i] = assert_cast *>(nullable_col->getNestedColumnPtr().get()); + } + else + { + null_map_cols[i] = nullptr; + data_cols[i] = assert_cast *>(instructions[i].source.get()); + } + } + + for (size_t row_i = 0; row_i < rows; ++row_i) + { + auto & instruction = instructions[inserts[row_i]]; + size_t index = instruction.source_is_constant ? 0 : row_i; + res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; + null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; + } + } + template static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res) { diff --git a/tests/queries/0_stateless/02918_multif_for_nullable.sh b/tests/queries/0_stateless/02918_multif_for_nullable.sh new file mode 100644 index 00000000000..cd9ac8b904f --- /dev/null +++ b/tests/queries/0_stateless/02918_multif_for_nullable.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (d Nullable(Int64)) engine=Memory" +$CLICKHOUSE_CLIENT -q "insert into test_tbl select * from numbers(5)" +$CLICKHOUSE_CLIENT -q "select multiIf(d > 0, 1, -1), multiIf(d > 1, d-1, -1), multiIf(d > 2, null, -1) from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file From b36ead13b05613218be8136701ab640a8f7823e3 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 11 Dec 2023 19:54:39 +0800 Subject: [PATCH 028/116] improve multiIf for nullable --- .../queries/0_stateless/02918_multif_for_nullable.reference | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02918_multif_for_nullable.reference diff --git a/tests/queries/0_stateless/02918_multif_for_nullable.reference b/tests/queries/0_stateless/02918_multif_for_nullable.reference new file mode 100644 index 00000000000..f58086cfee1 --- /dev/null +++ b/tests/queries/0_stateless/02918_multif_for_nullable.reference @@ -0,0 +1,5 @@ +-1 -1 -1 +1 -1 -1 +1 1 -1 +1 2 \N +1 3 \N From 46166c43bc7b289bc60346b7e00a5dc133b0f38f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 11 Dec 2023 20:18:53 +0800 Subject: [PATCH 029/116] some modify --- src/Functions/multiIf.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 8363768c0fb..d299e2b66ce 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -419,7 +419,7 @@ private: auto & instruction = instructions[inserts[row_i]]; size_t index = instruction.source_is_constant ? 0 : row_i; res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; - null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; + null_map_data[row_i] = null_map_cols[inserts[row_i]] ? null_map_cols[inserts[row_i]]->getData()[index] : 0; } } From 89a733a2f5696c2988d50ffe2a165110adb43414 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 11 Dec 2023 20:40:47 +0800 Subject: [PATCH 030/116] change the permission --- tests/queries/0_stateless/02918_multif_for_nullable.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02918_multif_for_nullable.sh diff --git a/tests/queries/0_stateless/02918_multif_for_nullable.sh b/tests/queries/0_stateless/02918_multif_for_nullable.sh old mode 100644 new mode 100755 From e29b78d20b7db6a7cf38325bfe0f5353680988cc Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 13 Dec 2023 16:06:24 +0800 Subject: [PATCH 031/116] improve by use memcpy --- src/Core/Settings.h | 4 +- src/Functions/multiIf.cpp | 87 +++++++++++++++++++++++++++++++++------ 2 files changed, 77 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fae2e3ebf89..cb16e695734 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -521,7 +521,9 @@ class IColumn; M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' print/parse the month name instead of minutes.", 0) \ M(Bool, parsedatetime_parse_without_leading_zeros, true, "Formatters '%c', '%l' and '%k' in function 'parseDateTime()' parse months and hours without leading zeros.", 0) \ M(Bool, formatdatetime_format_without_leading_zeros, false, "Formatters '%c', '%l' and '%k' in function 'formatDateTime()' print months and hours without leading zeros.", 0) \ - \ + M(Bool, allow_execute_multiif_columnar_by_memcpy, false, "Allow execute multiIf function columnar by memcpy", 0) \ + M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \ + M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ M(Bool, throw_on_max_partitions_per_insert_block, true, "Used with max_partitions_per_insert_block. If true (default), an exception will be thrown when max_partitions_per_insert_block is reached. If false, details of the insert query reaching this limit with the number of partitions will be logged. This can be useful if you're trying to understand the impact on users when changing max_partitions_per_insert_block.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index d299e2b66ce..b68a02494a4 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -256,13 +256,13 @@ public: MutableColumnPtr res = ColumnVector::create(rows); \ if (!result_type->isNullable()) \ { \ - executeInstructionsColumnar(instructions, rows, res); \ + executeInstructionsColumnar(instructions, rows, res, settings.allow_execute_multiif_columnar_by_memcpy); \ return std::move(res); \ } \ else \ { \ MutableColumnPtr null_map = ColumnUInt8::create(rows); \ - executeInstructionsColumnarForNullable(instructions, rows, res, null_map); \ + executeInstructionsColumnarForNullable(instructions, rows, res, null_map, settings.allow_execute_multiif_columnar_by_memcpy); \ return ColumnNullable::create(std::move(res), std::move(null_map)); \ } \ } @@ -383,7 +383,8 @@ private: } template - static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map) + static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, + const MutableColumnPtr & res, const MutableColumnPtr & null_map, bool execute_by_memcpy) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); @@ -414,27 +415,87 @@ private: } } - for (size_t row_i = 0; row_i < rows; ++row_i) + if (!execute_by_memcpy) { - auto & instruction = instructions[inserts[row_i]]; - size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; - null_map_data[row_i] = null_map_cols[inserts[row_i]] ? null_map_cols[inserts[row_i]]->getData()[index] : 0; + for (size_t row_i = 0; row_i < rows; ++row_i) + { + auto & instruction = instructions[inserts[row_i]]; + size_t index = instruction.source_is_constant ? 0 : row_i; + res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; + null_map_data[row_i] = null_map_cols[inserts[row_i]] ? null_map_cols[inserts[row_i]]->getData()[index] : 0; + } + } + else + { + size_t insert_start_pos = 0; + for (size_t row_i = 0; row_i < rows; ++row_i) + { + S curr_insert = inserts[row_i]; + if (row_i != rows -1 && curr_insert == inserts[row_i + 1]) + { + continue; + } + else + { + if (instructions[curr_insert].source_is_constant) + { + for (size_t i = insert_start_pos; i <= row_i; ++i) + { + res_data[i] = data_cols[curr_insert]->getData()[0]; + null_map_data[i] = null_map_cols[curr_insert]->getData()[0]; + } + } + else + { + memcpy(res_data.data() + insert_start_pos, + data_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(T) * (row_i + 1 - insert_start_pos)); + memcpy(null_map_data.data() + insert_start_pos, + null_map_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(UInt8) * (row_i + 1 - insert_start_pos)); + } + insert_start_pos = row_i + 1; + } + } } } template - static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res) + static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, bool execute_by_memcpy) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); PaddedPODArray & res_data = assert_cast &>(*res).getData(); - for (size_t row_i = 0; row_i < rows; ++row_i) + if (!execute_by_memcpy) { - auto & instruction = instructions[inserts[row_i]]; - auto ref = instruction.source->getDataAt(row_i); - res_data[row_i] = *reinterpret_cast(ref.data); + for (size_t row_i = 0; row_i < rows; ++row_i) + { + auto & instruction = instructions[inserts[row_i]]; + auto ref = instruction.source->getDataAt(row_i); + res_data[row_i] = *reinterpret_cast(ref.data); + } + } + else + { + size_t insert_start_pos = 0; + std::vector *> data_cols(instructions.size()); + for (size_t i = 0; i < instructions.size(); ++i) + { + data_cols[i] = assert_cast *>(instructions[i].source.get()); + } + for (size_t row_i = 0; row_i < rows; ++row_i) + { + S curr_insert = inserts[row_i]; + if (row_i != rows -1 && curr_insert == inserts[row_i + 1]) + { + continue; + } + else + { + memcpy(res_data.data() + insert_start_pos, + data_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(T) * (row_i + 1 - insert_start_pos)); + insert_start_pos = row_i + 1; + } + } } } From 21b86c0650b4589d24899db29c79ff0e06460328 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 14 Dec 2023 19:33:15 +0800 Subject: [PATCH 032/116] improve --- src/Core/Settings.h | 1 + src/Functions/multiIf.cpp | 97 ++++++++++++++++++++------------------- 2 files changed, 52 insertions(+), 46 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cb16e695734..db526e09f9b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -522,6 +522,7 @@ class IColumn; M(Bool, parsedatetime_parse_without_leading_zeros, true, "Formatters '%c', '%l' and '%k' in function 'parseDateTime()' parse months and hours without leading zeros.", 0) \ M(Bool, formatdatetime_format_without_leading_zeros, false, "Formatters '%c', '%l' and '%k' in function 'formatDateTime()' print months and hours without leading zeros.", 0) \ M(Bool, allow_execute_multiif_columnar_by_memcpy, false, "Allow execute multiIf function columnar by memcpy", 0) \ + M(Float, skew_threshold_use_memcpy_execute_multiif_columnar, 0.9f, "The condition skew threshold to use memcpy while execute mulitiif columnar", 0) \ M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \ M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index b68a02494a4..87f63316732 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -35,13 +35,13 @@ namespace /// /// Additionally the arguments, conditions or branches, support nullable types /// and the NULL value, with a NULL condition treated as false. -class FunctionMultiIf final : public FunctionIfBase +class FunctionMultiIf : public FunctionIfBase { public: static constexpr auto name = "multiIf"; static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - explicit FunctionMultiIf(ContextPtr context_) : context(context_) { } + explicit FunctionMultiIf(ContextPtr context_) : context(context_) {} String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -143,7 +143,6 @@ public: * depending on values of conditions. */ - std::vector instructions; instructions.reserve(arguments.size() / 2 + 1); @@ -256,13 +255,15 @@ public: MutableColumnPtr res = ColumnVector::create(rows); \ if (!result_type->isNullable()) \ { \ - executeInstructionsColumnar(instructions, rows, res, settings.allow_execute_multiif_columnar_by_memcpy); \ + executeInstructionsColumnar(instructions, rows, res, instruction_use_memory_copy, \ + settings.skew_threshold_use_memcpy_execute_multiif_columnar); \ return std::move(res); \ } \ else \ { \ MutableColumnPtr null_map = ColumnUInt8::create(rows); \ - executeInstructionsColumnarForNullable(instructions, rows, res, null_map, settings.allow_execute_multiif_columnar_by_memcpy); \ + executeInstructionsColumnarForNullable(instructions, rows, res, null_map, instruction_use_memory_copy, \ + settings.skew_threshold_use_memcpy_execute_multiif_columnar); \ return ColumnNullable::create(std::move(res), std::move(null_map)); \ } \ } @@ -304,6 +305,8 @@ public: } private: + mutable std::optional instruction_use_memory_copy; + static void executeInstructions(std::vector & instructions, size_t rows, const MutableColumnPtr & res) { for (size_t i = 0; i < rows; ++i) @@ -382,9 +385,27 @@ private: } } + template + static void calculateWhetherUseMemoryCopy(PaddedPODArray & inserts, size_t instructions_size, std::optional & instruction_use_memory_copy, + double threshold_use_memcpy) + { + if (!instruction_use_memory_copy.has_value()) + { + std::vector instruction_insert_sizes(instructions_size); + for (size_t i = 0; i < inserts.size(); ++i) + instruction_insert_sizes[inserts[i]] += 1; + + for (size_t i = 0; i < instruction_insert_sizes.size(); ++i) + { + if (instruction_insert_sizes[i] * 1.0 / inserts.size() >= threshold_use_memcpy) + instruction_use_memory_copy.emplace(static_cast(i)); + } + } + } + template static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, - const MutableColumnPtr & res, const MutableColumnPtr & null_map, bool execute_by_memcpy) + const MutableColumnPtr & res, const MutableColumnPtr & null_map, std::optional & instruction_use_memory_copy, double threshold_use_memcpy) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); @@ -393,6 +414,8 @@ private: PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); std::vector *> data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); + calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); + for (size_t i = 0; i < instructions.size(); ++i) { if (instructions[i].source->isNullable()) @@ -410,63 +433,53 @@ private: } else { - null_map_cols[i] = nullptr; + null_map_cols[i] = ColumnUInt8::create(rows).get(); data_cols[i] = assert_cast *>(instructions[i].source.get()); } } - if (!execute_by_memcpy) + if (!instruction_use_memory_copy.has_value()) { + std::cout << "has no value 111" << std::endl; for (size_t row_i = 0; row_i < rows; ++row_i) { auto & instruction = instructions[inserts[row_i]]; size_t index = instruction.source_is_constant ? 0 : row_i; res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; - null_map_data[row_i] = null_map_cols[inserts[row_i]] ? null_map_cols[inserts[row_i]]->getData()[index] : 0; + null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; } } else { - size_t insert_start_pos = 0; + int val = instruction_use_memory_copy.value(); + memcpy(res_data.data(), data_cols[val]->getData().data(), sizeof(T) * rows); + memcpy(null_map_data.data(), null_map_cols[val]->getData().data(), sizeof(UInt8) * rows); for (size_t row_i = 0; row_i < rows; ++row_i) { - S curr_insert = inserts[row_i]; - if (row_i != rows -1 && curr_insert == inserts[row_i + 1]) + if (inserts[row_i] == val) { continue; } - else - { - if (instructions[curr_insert].source_is_constant) - { - for (size_t i = insert_start_pos; i <= row_i; ++i) - { - res_data[i] = data_cols[curr_insert]->getData()[0]; - null_map_data[i] = null_map_cols[curr_insert]->getData()[0]; - } - } - else - { - memcpy(res_data.data() + insert_start_pos, - data_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(T) * (row_i + 1 - insert_start_pos)); - memcpy(null_map_data.data() + insert_start_pos, - null_map_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(UInt8) * (row_i + 1 - insert_start_pos)); - } - insert_start_pos = row_i + 1; - } + auto & instruction = instructions[inserts[row_i]]; + size_t index = instruction.source_is_constant ? 0 : row_i; + res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; + null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; } } } template - static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, bool execute_by_memcpy) + static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, + std::optional & instruction_use_memory_copy, double threshold_use_memcpy) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); PaddedPODArray & res_data = assert_cast &>(*res).getData(); - if (!execute_by_memcpy) + calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); + if (!instruction_use_memory_copy.has_value()) { + std::cout << "has no value" << std::endl; for (size_t row_i = 0; row_i < rows; ++row_i) { auto & instruction = instructions[inserts[row_i]]; @@ -476,25 +489,17 @@ private: } else { - size_t insert_start_pos = 0; std::vector *> data_cols(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) - { data_cols[i] = assert_cast *>(instructions[i].source.get()); - } + + int val = instruction_use_memory_copy.value(); + memcpy(res_data.data(), data_cols[val]->getData().data(), sizeof(T) * rows); for (size_t row_i = 0; row_i < rows; ++row_i) { - S curr_insert = inserts[row_i]; - if (row_i != rows -1 && curr_insert == inserts[row_i + 1]) - { + if (inserts[row_i] == val) continue; - } - else - { - memcpy(res_data.data() + insert_start_pos, - data_cols[curr_insert]->getData().data() + insert_start_pos, sizeof(T) * (row_i + 1 - insert_start_pos)); - insert_start_pos = row_i + 1; - } + res_data[row_i] = data_cols[inserts[row_i]]->getData()[row_i]; } } } From d4a33bc0d1800088de7655a0f2c2c4dbc02ece44 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 14 Dec 2023 20:05:46 +0800 Subject: [PATCH 033/116] review fix --- src/Functions/multiIf.cpp | 38 +++++++++++++++++++------------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 87f63316732..7b14fd75c23 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -412,8 +412,8 @@ private: PaddedPODArray & res_data = assert_cast &>(*res).getData(); PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); - std::vector *> data_cols(instructions.size()); - std::vector null_map_cols(instructions.size()); + std::vector data_cols(instructions.size()); + std::vector null_map_cols(instructions.size()); calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); for (size_t i = 0; i < instructions.size(); ++i) @@ -425,35 +425,34 @@ private: nullable_col = assert_cast(instructions[i].source.get()); else { - const ColumnPtr data_column = assert_cast(instructions[i].source.get())->getDataColumnPtr(); + const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); nullable_col = assert_cast(data_column.get()); } - null_map_cols[i] = assert_cast(nullable_col->getNullMapColumnPtr().get()); - data_cols[i] = assert_cast *>(nullable_col->getNestedColumnPtr().get()); + null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); + data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); } else { - null_map_cols[i] = ColumnUInt8::create(rows).get(); - data_cols[i] = assert_cast *>(instructions[i].source.get()); + null_map_cols[i] = ColumnUInt8::create(rows)->getData().data(); + data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } if (!instruction_use_memory_copy.has_value()) { - std::cout << "has no value 111" << std::endl; for (size_t row_i = 0; row_i < rows; ++row_i) { auto & instruction = instructions[inserts[row_i]]; size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; - null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; + res_data[row_i] = *(data_cols[inserts[row_i]] + index); + null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); } } else { int val = instruction_use_memory_copy.value(); - memcpy(res_data.data(), data_cols[val]->getData().data(), sizeof(T) * rows); - memcpy(null_map_data.data(), null_map_cols[val]->getData().data(), sizeof(UInt8) * rows); + memcpy(res_data.data(), data_cols[val], sizeof(T) * rows); + memcpy(null_map_data.data(), null_map_cols[val], sizeof(UInt8) * rows); for (size_t row_i = 0; row_i < rows; ++row_i) { if (inserts[row_i] == val) @@ -462,8 +461,8 @@ private: } auto & instruction = instructions[inserts[row_i]]; size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = data_cols[inserts[row_i]]->getData()[index]; - null_map_data[row_i] = null_map_cols[inserts[row_i]]->getData()[index]; + res_data[row_i] = *(data_cols[inserts[row_i]] + index); + null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); } } } @@ -479,7 +478,6 @@ private: calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); if (!instruction_use_memory_copy.has_value()) { - std::cout << "has no value" << std::endl; for (size_t row_i = 0; row_i < rows; ++row_i) { auto & instruction = instructions[inserts[row_i]]; @@ -489,17 +487,19 @@ private: } else { - std::vector *> data_cols(instructions.size()); + std::vector data_cols(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) - data_cols[i] = assert_cast *>(instructions[i].source.get()); + { + data_cols[i]= assert_cast &>(*instructions[i].source).getData().data(); + } int val = instruction_use_memory_copy.value(); - memcpy(res_data.data(), data_cols[val]->getData().data(), sizeof(T) * rows); + memcpy(res_data.data(), data_cols[val], sizeof(T) * rows); for (size_t row_i = 0; row_i < rows; ++row_i) { if (inserts[row_i] == val) continue; - res_data[row_i] = data_cols[inserts[row_i]]->getData()[row_i]; + res_data[row_i] = *(data_cols[inserts[row_i]] + row_i); } } } From 37fb8a4dd99c53d974df5450e1636975268480de Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 14 Dec 2023 20:17:07 +0800 Subject: [PATCH 034/116] remove unused code --- src/Core/Settings.h | 4 +--- src/Functions/multiIf.cpp | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index db526e09f9b..33876dc09a0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -521,10 +521,8 @@ class IColumn; M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' print/parse the month name instead of minutes.", 0) \ M(Bool, parsedatetime_parse_without_leading_zeros, true, "Formatters '%c', '%l' and '%k' in function 'parseDateTime()' parse months and hours without leading zeros.", 0) \ M(Bool, formatdatetime_format_without_leading_zeros, false, "Formatters '%c', '%l' and '%k' in function 'formatDateTime()' print months and hours without leading zeros.", 0) \ - M(Bool, allow_execute_multiif_columnar_by_memcpy, false, "Allow execute multiIf function columnar by memcpy", 0) \ + \ M(Float, skew_threshold_use_memcpy_execute_multiif_columnar, 0.9f, "The condition skew threshold to use memcpy while execute mulitiif columnar", 0) \ - M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \ - M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ M(Bool, throw_on_max_partitions_per_insert_block, true, "Used with max_partitions_per_insert_block. If true (default), an exception will be thrown when max_partitions_per_insert_block is reached. If false, details of the insert query reaching this limit with the number of partitions will be logged. This can be useful if you're trying to understand the impact on users when changing max_partitions_per_insert_block.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 7b14fd75c23..7f5adefe82c 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -35,13 +35,13 @@ namespace /// /// Additionally the arguments, conditions or branches, support nullable types /// and the NULL value, with a NULL condition treated as false. -class FunctionMultiIf : public FunctionIfBase +class FunctionMultiIf final : public FunctionIfBase { public: static constexpr auto name = "multiIf"; static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - explicit FunctionMultiIf(ContextPtr context_) : context(context_) {} + explicit FunctionMultiIf(ContextPtr context_) : context(context_) { } String getName() const override { return name; } bool isVariadic() const override { return true; } From ca49bc067087f40d7a292bba62a8286c2930512c Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 14 Dec 2023 20:47:30 +0800 Subject: [PATCH 035/116] check style fix --- src/Functions/multiIf.cpp | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 7f5adefe82c..1eedbf4a90c 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -394,7 +394,6 @@ private: std::vector instruction_insert_sizes(instructions_size); for (size_t i = 0; i < inserts.size(); ++i) instruction_insert_sizes[inserts[i]] += 1; - for (size_t i = 0; i < instruction_insert_sizes.size(); ++i) { if (instruction_insert_sizes[i] * 1.0 / inserts.size() >= threshold_use_memcpy) @@ -404,8 +403,8 @@ private: } template - static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, - const MutableColumnPtr & res, const MutableColumnPtr & null_map, std::optional & instruction_use_memory_copy, double threshold_use_memcpy) + static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, const MutableColumnPtr & res, + const MutableColumnPtr & null_map, std::optional & instruction_use_memory_copy, double threshold_use_memcpy) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); @@ -415,7 +414,6 @@ private: std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); - for (size_t i = 0; i < instructions.size(); ++i) { if (instructions[i].source->isNullable()) @@ -456,9 +454,7 @@ private: for (size_t row_i = 0; row_i < rows; ++row_i) { if (inserts[row_i] == val) - { continue; - } auto & instruction = instructions[inserts[row_i]]; size_t index = instruction.source_is_constant ? 0 : row_i; res_data[row_i] = *(data_cols[inserts[row_i]] + index); @@ -489,10 +485,7 @@ private: { std::vector data_cols(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) - { data_cols[i]= assert_cast &>(*instructions[i].source).getData().data(); - } - int val = instruction_use_memory_copy.value(); memcpy(res_data.data(), data_cols[val], sizeof(T) * rows); for (size_t row_i = 0; row_i < rows; ++row_i) From 380d9c996be740b819c64a12a5bb0c60cd1c6378 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 14 Dec 2023 22:54:43 +0800 Subject: [PATCH 036/116] consider constant --- src/Functions/multiIf.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 1eedbf4a90c..849ad384bcf 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -386,17 +386,17 @@ private: } template - static void calculateWhetherUseMemoryCopy(PaddedPODArray & inserts, size_t instructions_size, std::optional & instruction_use_memory_copy, + static void calculateWhetherUseMemoryCopy(PaddedPODArray & inserts, std::vector & instructions, std::optional & instruction_use_memory_copy, double threshold_use_memcpy) { if (!instruction_use_memory_copy.has_value()) { - std::vector instruction_insert_sizes(instructions_size); + std::vector instruction_insert_sizes(instructions.size()); for (size_t i = 0; i < inserts.size(); ++i) instruction_insert_sizes[inserts[i]] += 1; for (size_t i = 0; i < instruction_insert_sizes.size(); ++i) { - if (instruction_insert_sizes[i] * 1.0 / inserts.size() >= threshold_use_memcpy) + if (!instructions[i].source_is_constant && instruction_insert_sizes[i] * 1.0 / inserts.size() >= threshold_use_memcpy) instruction_use_memory_copy.emplace(static_cast(i)); } } @@ -413,7 +413,7 @@ private: PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); - calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); + calculateWhetherUseMemoryCopy(inserts, instructions, instruction_use_memory_copy, threshold_use_memcpy); for (size_t i = 0; i < instructions.size(); ++i) { if (instructions[i].source->isNullable()) @@ -471,7 +471,7 @@ private: calculateInserts(instructions, rows, inserts); PaddedPODArray & res_data = assert_cast &>(*res).getData(); - calculateWhetherUseMemoryCopy(inserts, instructions.size(), instruction_use_memory_copy, threshold_use_memcpy); + calculateWhetherUseMemoryCopy(inserts, instructions, instruction_use_memory_copy, threshold_use_memcpy); if (!instruction_use_memory_copy.has_value()) { for (size_t row_i = 0; row_i < rows; ++row_i) From 3dc8f5ad52dc39a52bfc440cf59085b1eb3681fd Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 18 Dec 2023 12:02:40 +0800 Subject: [PATCH 037/116] improve multiIF --- src/Core/Settings.h | 1 - src/Functions/multiIf.cpp | 129 +++++++++----------------------------- 2 files changed, 31 insertions(+), 99 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 33876dc09a0..fae2e3ebf89 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -522,7 +522,6 @@ class IColumn; M(Bool, parsedatetime_parse_without_leading_zeros, true, "Formatters '%c', '%l' and '%k' in function 'parseDateTime()' parse months and hours without leading zeros.", 0) \ M(Bool, formatdatetime_format_without_leading_zeros, false, "Formatters '%c', '%l' and '%k' in function 'formatDateTime()' print months and hours without leading zeros.", 0) \ \ - M(Float, skew_threshold_use_memcpy_execute_multiif_columnar, 0.9f, "The condition skew threshold to use memcpy while execute mulitiif columnar", 0) \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ M(Bool, throw_on_max_partitions_per_insert_block, true, "Used with max_partitions_per_insert_block. If true (default), an exception will be thrown when max_partitions_per_insert_block is reached. If false, details of the insert query reaching this limit with the number of partitions will be logged. This can be useful if you're trying to understand the impact on users when changing max_partitions_per_insert_block.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 849ad384bcf..cdc64874d3d 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -253,19 +253,12 @@ public: if (which.is##TYPE()) \ { \ MutableColumnPtr res = ColumnVector::create(rows); \ + MutableColumnPtr null_map = ColumnUInt8::create(rows); \ + executeInstructionsColumnar(instructions, rows, res, null_map, result_type->isNullable()); \ if (!result_type->isNullable()) \ - { \ - executeInstructionsColumnar(instructions, rows, res, instruction_use_memory_copy, \ - settings.skew_threshold_use_memcpy_execute_multiif_columnar); \ return std::move(res); \ - } \ else \ - { \ - MutableColumnPtr null_map = ColumnUInt8::create(rows); \ - executeInstructionsColumnarForNullable(instructions, rows, res, null_map, instruction_use_memory_copy, \ - settings.skew_threshold_use_memcpy_execute_multiif_columnar); \ return ColumnNullable::create(std::move(res), std::move(null_map)); \ - } \ } #define ENUMERATE_NUMERIC_TYPES(M, INDEX) \ @@ -305,7 +298,6 @@ public: } private: - mutable std::optional instruction_use_memory_copy; static void executeInstructions(std::vector & instructions, size_t rows, const MutableColumnPtr & res) { @@ -385,94 +377,16 @@ private: } } - template - static void calculateWhetherUseMemoryCopy(PaddedPODArray & inserts, std::vector & instructions, std::optional & instruction_use_memory_copy, - double threshold_use_memcpy) - { - if (!instruction_use_memory_copy.has_value()) - { - std::vector instruction_insert_sizes(instructions.size()); - for (size_t i = 0; i < inserts.size(); ++i) - instruction_insert_sizes[inserts[i]] += 1; - for (size_t i = 0; i < instruction_insert_sizes.size(); ++i) - { - if (!instructions[i].source_is_constant && instruction_insert_sizes[i] * 1.0 / inserts.size() >= threshold_use_memcpy) - instruction_use_memory_copy.emplace(static_cast(i)); - } - } - } - - template - static void executeInstructionsColumnarForNullable(std::vector & instructions, size_t rows, const MutableColumnPtr & res, - const MutableColumnPtr & null_map, std::optional & instruction_use_memory_copy, double threshold_use_memcpy) + template + static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map, + bool nullable) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); PaddedPODArray & res_data = assert_cast &>(*res).getData(); PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); - std::vector data_cols(instructions.size()); - std::vector null_map_cols(instructions.size()); - calculateWhetherUseMemoryCopy(inserts, instructions, instruction_use_memory_copy, threshold_use_memcpy); - for (size_t i = 0; i < instructions.size(); ++i) - { - if (instructions[i].source->isNullable()) - { - const ColumnNullable * nullable_col; - if (!instructions[i].source_is_constant) - nullable_col = assert_cast(instructions[i].source.get()); - else - { - const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); - nullable_col = assert_cast(data_column.get()); - } - null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); - data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); - } - else - { - null_map_cols[i] = ColumnUInt8::create(rows)->getData().data(); - data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); - } - } - - if (!instruction_use_memory_copy.has_value()) - { - for (size_t row_i = 0; row_i < rows; ++row_i) - { - auto & instruction = instructions[inserts[row_i]]; - size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = *(data_cols[inserts[row_i]] + index); - null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); - } - } - else - { - int val = instruction_use_memory_copy.value(); - memcpy(res_data.data(), data_cols[val], sizeof(T) * rows); - memcpy(null_map_data.data(), null_map_cols[val], sizeof(UInt8) * rows); - for (size_t row_i = 0; row_i < rows; ++row_i) - { - if (inserts[row_i] == val) - continue; - auto & instruction = instructions[inserts[row_i]]; - size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = *(data_cols[inserts[row_i]] + index); - null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); - } - } - } - - template - static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, - std::optional & instruction_use_memory_copy, double threshold_use_memcpy) - { - PaddedPODArray inserts(rows, static_cast(instructions.size())); - calculateInserts(instructions, rows, inserts); - - PaddedPODArray & res_data = assert_cast &>(*res).getData(); - calculateWhetherUseMemoryCopy(inserts, instructions, instruction_use_memory_copy, threshold_use_memcpy); - if (!instruction_use_memory_copy.has_value()) + if (!nullable) { for (size_t row_i = 0; row_i < rows; ++row_i) { @@ -484,15 +398,34 @@ private: else { std::vector data_cols(instructions.size()); + std::vector null_map_cols(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) - data_cols[i]= assert_cast &>(*instructions[i].source).getData().data(); - int val = instruction_use_memory_copy.value(); - memcpy(res_data.data(), data_cols[val], sizeof(T) * rows); + { + if (instructions[i].source->isNullable()) + { + const ColumnNullable * nullable_col; + if (!instructions[i].source_is_constant) + nullable_col = assert_cast(instructions[i].source.get()); + else + { + const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); + nullable_col = assert_cast(data_column.get()); + } + null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); + data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); + } + else + { + null_map_cols[i] = ColumnUInt8::create(rows)->getData().data(); + data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); + } + } for (size_t row_i = 0; row_i < rows; ++row_i) { - if (inserts[row_i] == val) - continue; - res_data[row_i] = *(data_cols[inserts[row_i]] + row_i); + auto & instruction = instructions[inserts[row_i]]; + size_t index = instruction.source_is_constant ? 0 : row_i; + res_data[row_i] = *(data_cols[inserts[row_i]] + index); + null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); } } } From 372de8a04640675e170d6f7180d23e719d5074b0 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 20 Dec 2023 10:12:24 -0500 Subject: [PATCH 038/116] refine patch --- contrib/CMakeLists.txt | 4 +- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 149 +++++++++--------- .../sql-reference/statements/create/table.md | 2 +- src/Compression/CompressionCodecZSTD.cpp | 2 +- src/Compression/CompressionCodecZSTDQAT.cpp | 16 +- .../aspell-ignore/en/aspell-dict.txt | 4 +- 6 files changed, 89 insertions(+), 88 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 387bc84f126..07b7b410705 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -173,7 +173,7 @@ add_contrib (c-ares-cmake c-ares) if (OS_LINUX AND ARCH_AMD64 AND ENABLE_SSE42) option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES}) elseif(ENABLE_QPL) - message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 arch with SSE 4.2 or higher") + message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 with SSE 4.2 or higher") endif() if (ENABLE_QPL) add_contrib (idxd-config-cmake idxd-config) @@ -185,7 +185,7 @@ endif () if (OS_LINUX AND ARCH_AMD64) option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology (Intel® QATlib)" ${ENABLE_LIBRARIES}) elseif(ENABLE_QATLIB) - message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64 arch") + message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64") endif() if (ENABLE_QATLIB) option (ENABLE_QAT_USDM_DRIVER "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 43e085104a8..e5a275e92e0 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -1,90 +1,89 @@ # Intel® QuickAssist Technology ZSTD Plugin (QAT ZSTD Plugin) is a plugin to Zstandard*(ZSTD*) for accelerating compression by QAT. -# Out-of-tree means kernel don't have native support, user will build and install driver from external package: https://www.intel.com/content/www/us/en/download/765501.html +# ENABLE_QAT_OUT_OF_TREE_BUILD = 1 means kernel don't have native support, user will build and install driver from external package: https://www.intel.com/content/www/us/en/download/765501.html # meanwhile, user need to set ICP_ROOT environment variable which point to the root directory of QAT driver source tree. -# in-tree means kernel has built-in qat driver, QAT-ZSTD-PLUGIN just has dependency on qatlib. +# ENABLE_QAT_OUT_OF_TREE_BUILD = 0 means kernel has built-in qat driver, QAT-ZSTD-PLUGIN just has dependency on qatlib. if (ENABLE_QAT_OUT_OF_TREE_BUILD) -# Intel QATZSTD out-of-tree build -message(STATUS "Intel QATZSTD out-of-tree build, ICP_ROOT:${ICP_ROOT}") + message(STATUS "Intel QATZSTD out-of-tree build, ICP_ROOT:${ICP_ROOT}") -set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") -set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") -set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") -set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") -set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") -set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") -set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") -set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") -set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") -if (ENABLE_QAT_USDM_DRIVER) - add_definitions(-DENABLE_USDM_DRV) -endif() -add_library(_qatzstd_plugin ${QATZSTD_SRC}) -target_link_libraries (_qatzstd_plugin PUBLIC ${USDM_LIBRARY} ${QAT_S_LIBRARY}) -target_include_directories(_qatzstd_plugin - SYSTEM PUBLIC "${QATZSTD_SRC_DIR}" - PRIVATE ${QAT_INCLUDE_DIR} - ${QAT_DC_INCLUDE_DIR} - ${QAT_AL_INCLUDE_DIR} - ${QAT_USDM_INCLUDE_DIR} - ${ZSTD_LIBRARY_DIR}) -target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) -add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) + set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") + set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") + set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") + set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") + set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") + set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") + set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") + set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") + set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") + if (ENABLE_QAT_USDM_DRIVER) + add_definitions(-DENABLE_USDM_DRV) + endif() + add_library(_qatzstd_plugin ${QATZSTD_SRC}) + target_link_libraries (_qatzstd_plugin PUBLIC ${USDM_LIBRARY} ${QAT_S_LIBRARY}) + target_include_directories(_qatzstd_plugin + SYSTEM PUBLIC "${QATZSTD_SRC_DIR}" + PRIVATE ${QAT_INCLUDE_DIR} + ${QAT_DC_INCLUDE_DIR} + ${QAT_AL_INCLUDE_DIR} + ${QAT_USDM_INCLUDE_DIR} + ${ZSTD_LIBRARY_DIR}) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) + add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) else () -# Intel QATZSTD in-tree build -message(STATUS "Intel QATZSTD in-tree build") -set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") -set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") -set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") + # Intel QATZSTD in-tree build + message(STATUS "Intel QATZSTD in-tree build") + set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") + set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") + set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib") -# please download&build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html -set(ICP_ROOT "${ClickHouse_SOURCE_DIR}/contrib/qatlib") -set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") -set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") -set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") -set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") -set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") -set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") -set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib") -set(LIBQAT_HEADER_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") + # please download&build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html + set(ICP_ROOT "${ClickHouse_SOURCE_DIR}/contrib/qatlib") + set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include") + set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc") + set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include") + set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv") + set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so") + set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so") + set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib") + set(LIBQAT_HEADER_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") -file(MAKE_DIRECTORY - "${LIBQAT_HEADER_DIR}/qat" -) -file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/cpa.h" - DESTINATION "${LIBQAT_HEADER_DIR}/qat/" -) -file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/dc/cpa_dc.h" - DESTINATION "${LIBQAT_HEADER_DIR}/qat/" -) -file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_poll.h" - DESTINATION "${LIBQAT_HEADER_DIR}/qat/" -) -file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_user.h" - DESTINATION "${LIBQAT_HEADER_DIR}/qat/" -) -file(COPY "${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv/qae_mem.h" - DESTINATION "${LIBQAT_HEADER_DIR}/qat/" -) + file(MAKE_DIRECTORY + "${LIBQAT_HEADER_DIR}/qat" + ) + file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/cpa.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" + ) + file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/dc/cpa_dc.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" + ) + file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_poll.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" + ) + file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_user.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" + ) + file(COPY "${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv/qae_mem.h" + DESTINATION "${LIBQAT_HEADER_DIR}/qat/" + ) -if (ENABLE_QAT_USDM_DRIVER) - add_definitions(-DENABLE_USDM_DRV) -endif() + if (ENABLE_QAT_USDM_DRIVER) + add_definitions(-DENABLE_USDM_DRV) + endif() -add_library(_qatzstd_plugin ${QATZSTD_SRC}) -target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qatlib ch_contrib::usdm) -target_include_directories(_qatzstd_plugin PRIVATE - ${QAT_INCLUDE_DIR} - ${QAT_DC_INCLUDE_DIR} - ${QAT_AL_INCLUDE_DIR} - ${QAT_USDM_INCLUDE_DIR} - ${ZSTD_LIBRARY_DIR} - ${LIBQAT_HEADER_DIR}) -target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION -DINTREE) -target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) -add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) + add_library(_qatzstd_plugin ${QATZSTD_SRC}) + target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qatlib ch_contrib::usdm) + target_include_directories(_qatzstd_plugin PRIVATE + ${QAT_INCLUDE_DIR} + ${QAT_DC_INCLUDE_DIR} + ${QAT_AL_INCLUDE_DIR} + ${QAT_USDM_INCLUDE_DIR} + ${ZSTD_LIBRARY_DIR} + ${LIBQAT_HEADER_DIR}) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTDQAT_COMPRESSION -DINTREE) + target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) + add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) endif () diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index d1c23c85c26..476cb551a7b 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -380,7 +380,7 @@ High compression levels are useful for asymmetric scenarios, like compress once, `ZSTDQAT[(level)]` — ZSTD QAT (hardware-accelerated compression) implemented by [QAT-ZSTD-Plugin](https://github.com/intel/QAT-ZSTD-Plugin) with configurable level. Default level: 1. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[6, 12\]. -ZSTDQAT works best if the system has Intel® QAT ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)) offloading device. If no device available, it will fallback to native ZSTD compression. +ZSTDQAT tries to use an Intel® QAT offloading device ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)). If no such device was found, it will fallback to ZSTD compression in software. #### DEFLATE_QPL diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index b7a790b2426..5b7a8809bbc 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -10,7 +11,6 @@ #include #include #include -#include "CompressionCodecZSTD.h" namespace DB { diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 376726b6195..92038fb78bc 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,5 +1,5 @@ #ifdef ENABLE_ZSTDQAT_COMPRESSION -#include +#include #include #include #include @@ -23,6 +23,7 @@ namespace ErrorCodes class CompressionCodecZSTDQAT : public CompressionCodecZSTD { public: + /// QAT Hardware only supports compression levels L1 to L12 static constexpr auto ZSTDQAT_SUPPORTED_MIN_LEVEL = 1; static constexpr auto ZSTDQAT_SUPPORTED_MAX_LEVEL = 12; explicit CompressionCodecZSTDQAT(int level_); @@ -44,17 +45,17 @@ UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 sourc if (!initialized) { cctx = ZSTD_createCCtx(); - /* Start QAT device, start QAT device at any time before compression job started */ + /// Start QAT device, start QAT device at any time before compression job started int res = QZSTD_startQatDevice(); - /* Create sequence producer state for QAT sequence producer */ + /// Create sequence producer state for QAT sequence producer sequenceProducerState = QZSTD_createSeqProdState(); - /* register qatSequenceProducer */ + /// register qatSequenceProducer ZSTD_registerSequenceProducer( cctx, sequenceProducerState, qatSequenceProducer ); - /* Enable sequence producer fallback */ + /// Enable sequence producer fallback ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); initialized = true; @@ -104,9 +105,10 @@ CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() { if (initialized) { - /* Free sequence producer state */ + /// Free sequence producer state QZSTD_freeSeqProdState(sequenceProducerState); - ZSTD_freeCCtx(cctx); + if (auto status = ZSTD_freeCCtx(cctx); status != 0) + LOG_WARNING(log, "ZSTD_freeCCtx failed with status: {} ", static_cast(status)); } } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 936bb4aa5d5..fdc58e3bd95 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -726,6 +726,7 @@ QueryCacheHits QueryCacheMisses QueryPreempted QueryThread +QuickAssist QuoteMeta RBAC RClickHouse @@ -996,6 +997,7 @@ ZooKeeperRequest ZooKeeperSession ZooKeeperWatch ZooKeepers +ZSTDQAT aarch accurateCast accurateCastOrDefault @@ -2662,5 +2664,3 @@ znode znodes zookeeperSessionUptime zstd -QuickAssist -ZSTDQAT From b1a7cae515b983f33934ebf893d66d1f63739086 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 21 Dec 2023 13:40:29 -0500 Subject: [PATCH 039/116] refine patch --- src/Compression/CompressionCodecZSTDQAT.cpp | 54 +++++++++++---------- 1 file changed, 28 insertions(+), 26 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 92038fb78bc..572329d874a 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -30,37 +30,18 @@ public: ~CompressionCodecZSTDQAT() override; protected: + /// TODO: So far, QAT hardware only support compression. For next generation in future, it will support decompression as well. UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; private: const int level; - mutable bool initialized; - mutable ZSTD_CCtx* cctx; - mutable void *sequenceProducerState; + ZSTD_CCtx * cctx; + void * sequenceProducerState; Poco::Logger * log; }; UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 source_size, char * dest) const { - if (!initialized) - { - cctx = ZSTD_createCCtx(); - /// Start QAT device, start QAT device at any time before compression job started - int res = QZSTD_startQatDevice(); - /// Create sequence producer state for QAT sequence producer - sequenceProducerState = QZSTD_createSeqProdState(); - /// register qatSequenceProducer - ZSTD_registerSequenceProducer( - cctx, - sequenceProducerState, - qatSequenceProducer - ); - /// Enable sequence producer fallback - ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); - initialized = true; - LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); - } size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); if (ZSTD_isError(compressed_size)) @@ -96,19 +77,40 @@ void registerCodecZSTDQAT(CompressionCodecFactory & factory) } CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) - : CompressionCodecZSTD(level_), level(level_), initialized(false), log(&Poco::Logger::get("CompressionCodecZSTDQAT")) + : CompressionCodecZSTD(level_), level(level_), log(&Poco::Logger::get("CompressionCodecZSTDQAT")) { setCodecDescription("ZSTDQAT", {std::make_shared(static_cast(level))}); + cctx = ZSTD_createCCtx(); + /// Start QAT device, start QAT device at any time before compression job started + int res = QZSTD_startQatDevice(); + LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); + /// Create sequence producer state for QAT sequence producer + sequenceProducerState = QZSTD_createSeqProdState(); + /// register qatSequenceProducer + ZSTD_registerSequenceProducer( + cctx, + sequenceProducerState, + qatSequenceProducer + ); + /// Enable sequence producer fallback + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); } CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() { - if (initialized) + /// Free sequence producer state + if (sequenceProducerState != nullptr) { - /// Free sequence producer state QZSTD_freeSeqProdState(sequenceProducerState); - if (auto status = ZSTD_freeCCtx(cctx); status != 0) + sequenceProducerState = nullptr; + } + if (cctx != nullptr) + { + auto status = ZSTD_freeCCtx(cctx); + if (status != 0) LOG_WARNING(log, "ZSTD_freeCCtx failed with status: {} ", static_cast(status)); + cctx = nullptr; } } From aaf98f0ed720527546726241389c808cd3356fa8 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 22 Dec 2023 14:24:15 -0500 Subject: [PATCH 040/116] refine setting with enable_zstd_qat_codec --- .../sql-reference/statements/create/table.md | 3 +- src/Client/Connection.cpp | 8 ++- src/Compression/CompressionCodecZSTDQAT.cpp | 1 + src/Compression/CompressionFactory.h | 4 +- .../CompressionFactoryAdditions.cpp | 14 ++++-- src/Compression/ICompressionCodec.h | 3 ++ src/Core/Settings.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/AlterCommands.cpp | 8 +-- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- tests/ci/stress.py | 1 + .../configs/enable_zstdqat_codec.xml | 7 +++ .../00804_test_zstdqat_codec_compression.sql | 49 +++++++++++++++++++ 16 files changed, 93 insertions(+), 17 deletions(-) create mode 100644 tests/integration/test_non_default_compression/configs/enable_zstdqat_codec.xml create mode 100644 tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 476cb551a7b..9b4d11fda46 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -380,7 +380,8 @@ High compression levels are useful for asymmetric scenarios, like compress once, `ZSTDQAT[(level)]` — ZSTD QAT (hardware-accelerated compression) implemented by [QAT-ZSTD-Plugin](https://github.com/intel/QAT-ZSTD-Plugin) with configurable level. Default level: 1. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[6, 12\]. -ZSTDQAT tries to use an Intel® QAT offloading device ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)). If no such device was found, it will fallback to ZSTD compression in software. +- ZSTDQAT is disabled by default and can only be used after setting configuration parameter `enable_zstd_qat_codec = 1`. +- ZSTDQAT tries to use an Intel® QAT offloading device ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)). If no such device was found, it will fallback to ZSTD compression in software. #### DEFLATE_QPL diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 75ca66f2647..352d2a53892 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -651,7 +651,13 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodec( + method, + level, + !settings->allow_suspicious_codecs, + settings->allow_experimental_codecs, + settings->enable_deflate_qpl_codec, + settings->enable_zstd_qat_codec); compression_codec = CompressionCodecFactory::instance().get(method, level); } else diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 572329d874a..8760f60cd69 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -30,6 +30,7 @@ public: ~CompressionCodecZSTDQAT() override; protected: + bool isZSTDQAT() const override { return true; } /// TODO: So far, QAT hardware only support compression. For next generation in future, it will support decompression as well. UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 4f2627587a3..e71476d564d 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -40,10 +40,10 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec, bool enable_zstd_qat_codec) const; /// Validate codecs AST specified by user - void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const; + void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec, bool enable_zstd_qat_codec) const; /// Get codec by AST and possible column_type. Some codecs can use /// information about type to improve inner settings, but every codec should diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index 98e9e7480da..a3612ccb4b2 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec, bool enable_zstd_qat_codec) const { if (family_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty"); @@ -43,13 +43,13 @@ void CompressionCodecFactory::validateCodec( { auto literal = std::make_shared(static_cast(*level)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec); + {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec); + {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); } } @@ -77,7 +77,7 @@ bool innerDataTypeIsFloat(const DataTypePtr & type) } ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const + const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec, bool enable_zstd_qat_codec) const { if (const auto * func = ast->as()) { @@ -165,6 +165,12 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'enable_deflate_qpl_codec' setting.", codec_family_name); + if (!enable_zstd_qat_codec && result_codec->isZSTDQAT()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Codec {} is disabled by default." + " You can enable it with the 'enable_zstd_qat_codec' setting.", + codec_family_name); + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index ca794511268..ebf8591d222 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -121,6 +121,9 @@ public: /// Is this the DEFLATE_QPL codec? virtual bool isDeflateQpl() const { return false; } + /// Is this the ZSTDQAT codec? + virtual bool isZSTDQAT() const { return false; } + /// If it does nothing. virtual bool isNone() const { return false; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7750bd4a092..64dd18e3592 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -348,6 +348,7 @@ class IColumn; M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(Bool, enable_deflate_qpl_codec, false, "Enable/disable the DEFLATE_QPL codec.", 0) \ + M(Bool, enable_zstd_qat_codec, false, "Enable/disable the ZSTDQAT codec.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 2b60b0b7b47..7575fe10785 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -595,6 +595,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs; bool enable_deflate_qpl_codec = attach || context_->getSettingsRef().enable_deflate_qpl_codec; + bool enable_zstd_qat_codec = attach || context_->getSettingsRef().enable_zstd_qat_codec; ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); @@ -655,7 +656,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.default_specifier == "ALIAS") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec); + col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); } if (col_decl.stat_type) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f2d7a58119f..622d9d91105 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2023,7 +2023,7 @@ void TCPHandler::initBlockOutput(const Block & block) if (state.compression == Protocol::Compression::Enable) { - CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_deflate_qpl_codec, query_settings.enable_zstd_qat_codec); state.maybe_compressed_out = std::make_shared( *out, CompressionCodecFactory::instance().get(method, level)); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d19f92ae767..d5e0e1eb236 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -431,7 +431,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) column.comment = *comment; if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true, true); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true, true, true); column.ttl = ttl; @@ -496,7 +496,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) else { if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true, true); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true, true, true); if (comment) column.comment = *comment; @@ -1237,7 +1237,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "this column name is reserved for _block_number persisting feature", backQuote(column_name)); if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec, context->getSettingsRef().enable_zstd_qat_codec); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -1262,7 +1262,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.hasAlias(column_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec, context->getSettingsRef().enable_zstd_qat_codec); } auto column_default = all_columns.getDefault(column_name); if (column_default) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index cf8f341f9ff..94c918e565c 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -151,7 +151,7 @@ void ColumnDescription::readText(ReadBuffer & buf) comment = col_ast->comment->as().value.get(); if (col_ast->codec) - codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true); + codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true, true); if (col_ast->ttl) ttl = col_ast->ttl; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index acdc6f142a4..650539ef1e9 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -740,7 +740,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_deflate_qpl_codec, settings.enable_zstd_qat_codec); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f601fed06ac..db50aa8c63e 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -285,7 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( { result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); + ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec, context->getSettingsRef().enable_zstd_qat_codec); } } diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 51f8e7d3551..8ef72a242ff 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -21,6 +21,7 @@ def get_options(i: int, upgrade_check: bool) -> str: options.append(f'''--db-engine="Replicated('/test/db/test_{i}', 's1', 'r1')"''') client_options.append("allow_experimental_database_replicated=1") client_options.append("enable_deflate_qpl_codec=1") + client_options.append("enable_zstd_qat_codec=1") # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. diff --git a/tests/integration/test_non_default_compression/configs/enable_zstdqat_codec.xml b/tests/integration/test_non_default_compression/configs/enable_zstdqat_codec.xml new file mode 100644 index 00000000000..c686b37a537 --- /dev/null +++ b/tests/integration/test_non_default_compression/configs/enable_zstdqat_codec.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql b/tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql new file mode 100644 index 00000000000..e79b4fb06f5 --- /dev/null +++ b/tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql @@ -0,0 +1,49 @@ +--Tags: no-fasttest, no-cpu-aarch64, no-cpu-s390x +-- no-fasttest because ZSTDQAT isn't available in fasttest +-- no-cpu-aarch64 and no-cpu-s390x because ZSTDQAT is x86-only + +-- A bunch of random DDLs to test the ZSTDQAT codec. + +SET enable_zstd_qat_codec = 1; + +-- Suppress test failures because stderr contains warning "Initialization of hardware-assisted DeflateQpl failed, falling +-- back to software DeflateQpl coded." +SET send_logs_level = 'fatal'; + +DROP TABLE IF EXISTS compression_codec; + +CREATE TABLE compression_codec( + id UInt64 CODEC(ZSTDQAT), + data String CODEC(ZSTDQAT), + ddd Date CODEC(ZSTDQAT), + ddd32 Date32 CODEC(ZSTDQAT), + somenum Float64 CODEC(ZSTDQAT), + somestr FixedString(3) CODEC(ZSTDQAT), + othernum Int64 CODEC(ZSTDQAT), + somearray Array(UInt8) CODEC(ZSTDQAT), + somemap Map(String, UInt32) CODEC(ZSTDQAT), + sometuple Tuple(UInt16, UInt64) CODEC(ZSTDQAT), +) ENGINE = MergeTree() ORDER BY tuple(); + +SHOW CREATE TABLE compression_codec; + +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5, [1,2,3], map('k1',1,'k2',2), tuple(1,2)); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6, [4,5,6], map('k3',3,'k4',4), tuple(3,4)); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6)); + +SELECT * FROM compression_codec ORDER BY id; + +OPTIMIZE TABLE compression_codec FINAL; + +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8, [10,11,12], map('k7',7,'k8',8), tuple(7,8)); + +DETACH TABLE compression_codec; +ATTACH TABLE compression_codec; + +SELECT count(*) FROM compression_codec WHERE id = 2 GROUP BY id; + +INSERT INTO compression_codec SELECT 3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6) FROM system.numbers LIMIT 10000; + +SELECT count(*) FROM compression_codec WHERE id = 3 GROUP BY id; + +DROP TABLE IF EXISTS compression_codec; From d7d1541d8117f579ef7d8c9bdfff7da0e9df4b8b Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 27 Dec 2023 19:43:59 +0800 Subject: [PATCH 041/116] add performance test xml --- tests/performance/multiif.xml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 tests/performance/multiif.xml diff --git a/tests/performance/multiif.xml b/tests/performance/multiif.xml new file mode 100644 index 00000000000..ad56ab3f5f2 --- /dev/null +++ b/tests/performance/multiif.xml @@ -0,0 +1,8 @@ + + CREATE TABLE test_multiif_t(d Nullable(Int64)) ENGINE Memory + INSERT INTO test_multiif_t SELECT * from numbers(300000000) + + select count(1) from test_multiif_t where multiIf(d > 2, d-2, d > 1, d-1, d >0, d, 0) > 1 SETTINGS max_threads=1 + + DROP TABLE IF EXISTS test_multiif_t + From 41cdedbf26b531120b368f036e11462a86e484b9 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 28 Dec 2023 16:16:54 +0800 Subject: [PATCH 042/116] review --- src/Functions/multiIf.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index cdc64874d3d..4f1802dcebf 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -253,7 +253,7 @@ public: if (which.is##TYPE()) \ { \ MutableColumnPtr res = ColumnVector::create(rows); \ - MutableColumnPtr null_map = ColumnUInt8::create(rows); \ + MutableColumnPtr null_map = result_type->isNullable() ? ColumnUInt8::create(rows) : nullptr; \ executeInstructionsColumnar(instructions, rows, res, null_map, result_type->isNullable()); \ if (!result_type->isNullable()) \ return std::move(res); \ @@ -378,14 +378,12 @@ private: } template - static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map, - bool nullable) + static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map, bool nullable) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); PaddedPODArray & res_data = assert_cast &>(*res).getData(); - PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); if (!nullable) { for (size_t row_i = 0; row_i < rows; ++row_i) @@ -397,6 +395,7 @@ private: } else { + PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) From 842252bcadc986983eb85bc3f95bba2afc3b81f5 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 29 Dec 2023 09:32:31 +0800 Subject: [PATCH 043/116] review fix --- src/Functions/multiIf.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 4f1802dcebf..bd30f8a758e 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -415,7 +415,7 @@ private: } else { - null_map_cols[i] = ColumnUInt8::create(rows)->getData().data(); + null_map_cols[i] = (i == 0 ? ColumnUInt8::create(rows, 0)->getData().data() : null_map_cols[0]); data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } From 62139f0073b0194294293b3c9a9a463525139794 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 29 Dec 2023 12:19:56 +0000 Subject: [PATCH 044/116] fix recursion for finish --- src/Common/AsyncLoader.cpp | 39 ++++++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 7c7b057143b..33dc0d8cf33 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -569,23 +569,30 @@ void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::excepti // Update dependent jobs for (const auto & dpt : dependent) { - chassert(scheduled_jobs.contains(dpt)); // All dependent jobs must be scheduled - Info & dpt_info = scheduled_jobs[dpt]; - dpt_info.dependencies_left--; - if (!dpt_info.isBlocked()) - enqueue(dpt_info, dpt, lock); - - if (status != LoadStatus::OK) + if (auto dpt_info = scheduled_jobs.find(dpt); dpt_info != scheduled_jobs.end()) { - std::exception_ptr cancel; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - if (dpt->dependency_failure) - dpt->dependency_failure(dpt, job, cancel); - }); - // Recurse into dependent job if it should be canceled - if (cancel) - finish(dpt, LoadStatus::CANCELED, cancel, lock); + dpt_info->second.dependencies_left--; + if (!dpt_info->second.isBlocked()) + enqueue(dpt_info->second, dpt, lock); + + if (status != LoadStatus::OK) + { + std::exception_ptr cancel; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + if (dpt->dependency_failure) + dpt->dependency_failure(dpt, job, cancel); + }); + // Recurse into dependent job if it should be canceled + if (cancel) + finish(dpt, LoadStatus::CANCELED, cancel, lock); + } + } + else + { + // Job has already been canceled. Do not enter twice into the same job during finish recursion. + // This happens in {A<-B; A<-C; B<-D; C<-D} graph for D if A is failed or canceled. + chassert(status == LoadStatus::CANCELED); } } From d7aca766c23be45cbda3c68b726f0f36e0dc5dd2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 29 Dec 2023 14:27:01 +0100 Subject: [PATCH 045/116] Update src/Common/AsyncLoader.h --- src/Common/AsyncLoader.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 8de6a5e2615..85de309b153 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -9,7 +9,6 @@ #include #include #include -#include <__concepts/invocable.h> #include #include #include From 4f15031f79f4dba7a56c2446b4f15d3bd943f17c Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 29 Dec 2023 14:28:16 +0100 Subject: [PATCH 046/116] Update src/Common/AsyncLoader.cpp --- src/Common/AsyncLoader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 33dc0d8cf33..0e0fa25e7a1 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include From e2b43c2cfadf552f173f0103ac28101d3aed6a87 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 29 Dec 2023 14:29:30 +0100 Subject: [PATCH 047/116] Update src/Databases/DatabaseOrdinary.cpp --- src/Databases/DatabaseOrdinary.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 75b2a852c1c..63dc4f528b0 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -20,7 +20,6 @@ #include #include #include -#include "Common/AsyncLoader.h" #include #include #include From 733e68cfd5b1317449aa1f85ca66202d251b423c Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 2 Jan 2024 17:47:45 +0800 Subject: [PATCH 048/116] review fix --- src/Functions/multiIf.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index bd30f8a758e..07e9b440edd 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -398,7 +398,8 @@ private: PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); - for (size_t i = 0; i < instructions.size(); ++i) + int first_non_nullable_source = -1; + for (int i = 0; i < static_cast(instructions.size()); ++i) { if (instructions[i].source->isNullable()) { @@ -415,7 +416,11 @@ private: } else { - null_map_cols[i] = (i == 0 ? ColumnUInt8::create(rows, 0)->getData().data() : null_map_cols[0]); + if (first_non_nullable_source < 0) + { + first_non_nullable_source = i; + } + null_map_cols[i] = (i == first_non_nullable_source ? ColumnUInt8::create(rows, 0)->getData().data() : null_map_cols[first_non_nullable_source]); data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } From 5c221d123d70932c20678e5dee9d265044f2f326 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 7 Dec 2023 11:14:36 +0100 Subject: [PATCH 049/116] MaterializedMySQL: Introduce MySQL Binlog Client One binlog connection for many databases. Suggesting to disable this feature by default for now. It should be explicitly enabled by SETTINGS use_binlog_client=1. But if you would permanently enable it in MaterializedMySQLSettings, it should keep old behavior and all tests should pass too. 1. Introduced `IBinlog` and its impl to read the binlog events from socket - `BinlogFromSocket`, or file - `BinlogFromFile`. Based on prev impl of `EventBase` and the same old binlog parsers. It fully keeps BC with old version. Fixed `./check-mysql-binlog` to test new impl. 2. Introduced `BinlogEventsDispatcher`, it reads the event from the source `IBinlog` and sends it to currently attached `IBinlog` instances. 3. Introduced `BinlogClient`, which is used to group a list of `BinlogEventsDispatcher` by MySQL binlog connection which is defined by `user:password@host:port`. All dispatchers with the same binlog position should be merged to one. 4. Introduced `BinlogClientFactory`, which is a singleton and it is used to track all binlogs created over the instance. 5. Introduced `use_binlog_client` setting to `MaterializedMySQL`, which forces to reuse a `BinlogClient` if it already exists in `BinlogClientCatalog` or create new one. By default, it is disabled. 6. Introduced `max_bytes_in_binlog_queue` setting to define the limit of bytes in binlog's queue of events. If bytes in the queue increases this limit, `BinlogEventsDispatcher` will stop reading new events from source `IBinlog` until the space for new events will be freed. 7. Introduced `max_milliseconds_to_wait_in_binlog_queue` setting to define max ms to wait when the max bytes exceeded. 7. Introduced `max_milliseconds_to_wait_in_binlog_queue` setting to define max ms to wait when the max bytes exceeded. 8. Introduced `max_bytes_in_binlog_dispatcher_buffer` setting to define max bytes in the binlog dispatcher's buffer before it is flushed to attached binlogs. 9. Introduced `max_flush_milliseconds_in_binlog_dispatcher` setting to define max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlogs. 10. Introduced `system.mysql_binlogs` system table, which shows a list of active binlogs. 11. Introduced `UnparsedRowsEvent` and `MYSQL_UNPARSED_ROWS_EVENT`, which defines that an event is not parsed and should be explicitly parsed later. 12. Fixed bug when not possible to apply DDL since syntax error or unsupported SQL. @larspars is the author of following: `GTIDSets::contains()` `ReplicationHelper` `shouldReconnectOnException()` --- .../settings/mysql-binlog-client.md | 176 ++ src/Common/randomNumber.h | 12 + src/Core/MySQL/MySQLGtid.cpp | 42 + src/Core/MySQL/MySQLGtid.h | 27 + src/Core/MySQL/MySQLReplication.cpp | 63 +- src/Core/MySQL/MySQLReplication.h | 29 +- src/Core/MySQL/tests/gtest_MySQLGtid.cpp | 40 + .../MySQL/DatabaseMaterializedMySQL.cpp | 12 +- .../MySQL/DatabaseMaterializedMySQL.h | 2 + .../MySQL/MaterializedMySQLSettings.h | 5 + .../MySQL/MaterializedMySQLSyncThread.cpp | 255 ++- .../MySQL/MaterializedMySQLSyncThread.h | 17 +- src/Databases/MySQL/MySQLBinlog.cpp | 500 +++++ src/Databases/MySQL/MySQLBinlog.h | 120 ++ src/Databases/MySQL/MySQLBinlogClient.cpp | 104 + src/Databases/MySQL/MySQLBinlogClient.h | 57 + .../MySQL/MySQLBinlogClientFactory.cpp | 46 + .../MySQL/MySQLBinlogClientFactory.h | 38 + .../MySQL/MySQLBinlogEventsDispatcher.cpp | 626 ++++++ .../MySQL/MySQLBinlogEventsDispatcher.h | 136 ++ src/Databases/MySQL/tests/data/binlog.000016 | Bin 0 -> 1237 bytes src/Databases/MySQL/tests/data/binlog.001390 | Bin 0 -> 3091 bytes .../MySQL/tests/gtest_mysql_binlog.cpp | 1754 +++++++++++++++++ .../System/StorageSystemMySQLBinlogs.cpp | 164 ++ .../System/StorageSystemMySQLBinlogs.h | 29 + src/Storages/System/attachSystemTables.cpp | 7 + .../materialized_with_ddl.py | 695 +++++++ .../test_materialized_mysql_database/test.py | 158 ++ utils/check-mysql-binlog/main.cpp | 219 +- 29 files changed, 5134 insertions(+), 199 deletions(-) create mode 100644 docs/en/operations/settings/mysql-binlog-client.md create mode 100644 src/Common/randomNumber.h create mode 100644 src/Core/MySQL/tests/gtest_MySQLGtid.cpp create mode 100644 src/Databases/MySQL/MySQLBinlog.cpp create mode 100644 src/Databases/MySQL/MySQLBinlog.h create mode 100644 src/Databases/MySQL/MySQLBinlogClient.cpp create mode 100644 src/Databases/MySQL/MySQLBinlogClient.h create mode 100644 src/Databases/MySQL/MySQLBinlogClientFactory.cpp create mode 100644 src/Databases/MySQL/MySQLBinlogClientFactory.h create mode 100644 src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp create mode 100644 src/Databases/MySQL/MySQLBinlogEventsDispatcher.h create mode 100644 src/Databases/MySQL/tests/data/binlog.000016 create mode 100644 src/Databases/MySQL/tests/data/binlog.001390 create mode 100644 src/Databases/MySQL/tests/gtest_mysql_binlog.cpp create mode 100644 src/Storages/System/StorageSystemMySQLBinlogs.cpp create mode 100644 src/Storages/System/StorageSystemMySQLBinlogs.h diff --git a/docs/en/operations/settings/mysql-binlog-client.md b/docs/en/operations/settings/mysql-binlog-client.md new file mode 100644 index 00000000000..1e1a2449e1c --- /dev/null +++ b/docs/en/operations/settings/mysql-binlog-client.md @@ -0,0 +1,176 @@ +# The MySQL Binlog Client + +The MySQL Binlog Client provides a mechanism in ClickHouse to share the binlog from a MySQL instance among multiple [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) databases. This avoids consuming unnecessary bandwidth and CPU when replicating more than one schema/database. + +The implementation is resilient against crashes and disk issues. The executed GTID sets of the binlog itself and the consuming databases have persisted only after the data they describe has been safely persisted as well. The implementation also tolerates re-doing aborted operations (at-least-once delivery). + +# Settings + +## use_binlog_client + +Forces to reuse existing MySQL binlog connection or creates new one if does not exist. The connection is defined by `user:pass@host:port`. + +Default value: 0 + +**Example** + +```sql +-- create MaterializedMySQL databases that read the events from the binlog client +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +CREATE DATABASE db3 ENGINE = MaterializedMySQL('host:port', 'db3', 'user2', 'password2') SETTINGS use_binlog_client=1 +``` + +Databases `db1` and `db2` will use the same binlog connection, since they use the same `user:pass@host:port`. Database `db3` will use separate binlog connection. + +## max_bytes_in_binlog_queue + +Defines the limit of bytes in the events binlog queue. If bytes in the queue increases this limit, it will stop reading new events from MySQL until the space for new events will be freed. This introduces the memory limits. Very high value could consume all available memory. Very low value could make the databases to wait for new events. + +Default value: 67108864 + +**Example** + +```sql +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +``` + +If database `db1` is unable to consume binlog events fast enough and the size of the events queue exceeds `33554432` bytes, reading of new events from MySQL is postponed until `db1` +consumes the events and releases some space. + +NOTE: This will impact to `db2`, and it will be waiting for new events too, since they share the same connection. + +## max_milliseconds_to_wait_in_binlog_queue + +Defines the max milliseconds to wait when `max_bytes_in_binlog_queue` exceeded. After that it will detach the database from current binlog connection and will retry establish new one to prevent other databases to wait for this database. + +Default value: 10000 + +**Example** + +```sql +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432, max_milliseconds_to_wait_in_binlog_queue=1000 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +``` + +If the event queue of database `db1` is full, the binlog connection will be waiting in `1000`ms and if the database is not able to consume the events, it will be detached from the connection to create another one. + +NOTE: If the database `db1` has been detached from the shared connection and created new one, after the binlog connections for `db1` and `db2` have the same positions they will be merged to one. And `db1` and `db2` will use the same connection again. + +## max_bytes_in_binlog_dispatcher_buffer + +Defines the max bytes in the binlog dispatcher's buffer before it is flushed to attached binlog. The events from MySQL binlog connection are buffered before sending to attached databases. It increases the events throughput from the binlog to databases. + +Default value: 1048576 + +## max_flush_milliseconds_in_binlog_dispatcher + +Defines the max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlog. If there are no events received from MySQL binlog connection for a while, after some time buffered events should be sent to the attached databases. + +Default value: 1000 + +# Design + +## The Binlog Events Dispatcher + +Currently each MaterializedMySQL database opens its own connection to MySQL to subscribe to binlog events. There is a need to have only one connection and _dispatch_ the binlog events to all databases that replicate from the same MySQL instance. + +## Each MaterializedMySQL Database Has Its Own Event Queue + +To prevent slowing down other instances there should be an _event queue_ per MaterializedMySQL database to handle the events independently of the speed of other instances. The dispatcher reads an event from the binlog, and sends it to every MaterializedMySQL database that needs it. Each database handles its events in separate threads. + +## Catching up + +If several databases have the same binlog position, they can use the same dispatcher. If a newly created database (or one that has been detached for some time) requests events that have been already processed, we need to create another communication _channel_ to the binlog. We do this by creating another temporary dispatcher for such databases. When the new dispatcher _catches up with_ the old one, the new/temporary dispatcher is not needed anymore and all databases getting events from this dispatcher can be moved to the old one. + +## Memory Limit + +There is a _memory limit_ to control event queue memory consumption per MySQL Client. If a database is not able to handle events fast enough, and the event queue is getting full, we have the following options: + +1. The dispatcher is blocked until the slowest database frees up space for new events. All other databases are waiting for the slowest one. (Preferred) +2. The dispatcher is _never_ blocked, but suspends incremental sync for the slow database and continues dispatching events to remained databases. + +## Performance + +A lot of CPU can be saved by not processing every event in every database. The binlog contains events for all databases, it is wasteful to distribute row events to a database that it will not process it, especially if there are a lot of databases. This requires some sort of per-database binlog filtering and buffering. + +Currently all events are sent to all MaterializedMySQL databases but parsing the event which consumes CPU is up to the database. + +# Detailed Design + +1. If a client (e.g. database) wants to read a stream of the events from MySQL binlog, it creates a connection to remote binlog by host/user/password and _executed GTID set_ params. +2. If another client wants to read the events from the binlog but for different _executed GTID set_, it is **not** possible to reuse existing connection to MySQL, then need to create another connection to the same remote binlog. (_This is how it is implemented today_). +3. When these 2 connections get the same binlog positions, they read the same events. It is logical to drop duplicate connection and move all its users out. And now one connection dispatches binlog events to several clients. Obviously only connections to the same binlog should be merged. + +## Classes + +1. One connection can send (or dispatch) events to several clients and might be called `BinlogEventsDispatcher`. +2. Several dispatchers grouped by _user:password@host:port_ in `BinlogClient`. Since they point to the same binlog. +3. The clients should communicate only with public API from `BinlogClient`. The result of using `BinlogClient` is an object that implements `IBinlog` to read events from. This implementation of `IBinlog` must be compatible with old implementation `MySQLFlavor` -> when replacing old implementation by new one, the behavior must not be changed. + +## SQL + +```sql +-- create MaterializedMySQL databases that read the events from the binlog client +CREATE DATABASE db1_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=1024; +CREATE DATABASE db2_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db3_client1 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db4_client2 ENGINE = MaterializedMySQL('host2:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db5_client3 ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db6_old ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=0; +``` + +Databases `db1_client1`, `db2_client1` and `db3_client1` share one instance of `BinlogClient` since they have the same params. `BinlogClient` will create 3 connections to MySQL server thus 3 instances of `BinlogEventsDispatcher`, but if these connections would have the same binlog position, they should be merged to one connection. Means all clients will be moved to one dispatcher and others will be closed. Databases `db4_client2` and `db5_client3` would use 2 different independent `BinlogClient` instances. Database `db6_old` will use old implementation. NOTE: By default `use_binlog_client` is disabled. Setting `max_bytes_in_binlog_queue` defines the max allowed bytes in the binlog queue. By default, it is `1073741824` bytes. If number of bytes exceeds this limit, the dispatching will be stopped until the space will be freed for new events. + +## Binlog Table Structure + +To see the status of the all `BinlogClient` instances there is `system.mysql_binlogs` system table. It shows the list of all created and _alive_ `IBinlog` instances with information about its `BinlogEventsDispatcher` and `BinlogClient`. + +Example: + +``` +SELECT * FROM system.mysql_binlogs FORMAT Vertical +Row 1: +────── +binlog_client_name: root@127.0.0.1:3306 +name: test_Clickhouse1 +mysql_binlog_name: binlog.001154 +mysql_binlog_pos: 7142294 +mysql_binlog_timestamp: 1660082447 +mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304 +dispatcher_name: Applier +dispatcher_mysql_binlog_name: binlog.001154 +dispatcher_mysql_binlog_pos: 7142294 +dispatcher_mysql_binlog_timestamp: 1660082447 +dispatcher_mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304 +size: 0 +bytes: 0 +max_bytes: 0 +``` + +### Tests + +Unit tests: + +``` +$ ./unit_tests_dbms --gtest_filter=MySQLBinlog.* +``` + +Integration tests: + +``` +$ pytest -s -vv test_materialized_mysql_database/test.py::test_binlog_client +``` + +Dumps events from the file + +``` +$ ./utils/check-mysql-binlog/check-mysql-binlog --binlog binlog.001392 +``` + +Dumps events from the server + +``` +$ ./utils/check-mysql-binlog/check-mysql-binlog --host 127.0.0.1 --port 3306 --user root --password pass --gtid a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30462856 +``` diff --git a/src/Common/randomNumber.h b/src/Common/randomNumber.h new file mode 100644 index 00000000000..b795f32baca --- /dev/null +++ b/src/Common/randomNumber.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include + +inline UInt32 randomNumber() +{ + pcg64_fast rng{randomSeed()}; + std::uniform_int_distribution dist6( + std::numeric_limits::min(), std::numeric_limits::max()); + return static_cast(dist6(rng)); +} diff --git a/src/Core/MySQL/MySQLGtid.cpp b/src/Core/MySQL/MySQLGtid.cpp index 2b46c3d14ad..7916f882979 100644 --- a/src/Core/MySQL/MySQLGtid.cpp +++ b/src/Core/MySQL/MySQLGtid.cpp @@ -188,4 +188,46 @@ String GTIDSets::toPayload() const return buffer.str(); } +bool GTIDSet::contains(const GTIDSet & gtid_set) const +{ + //we contain the other set if each of its intervals are contained in any of our intervals. + //use the fact that intervals are sorted to make this linear instead of quadratic. + if (uuid != gtid_set.uuid) { return false; } + + auto mine = intervals.begin(), other = gtid_set.intervals.begin(); + auto my_end = intervals.end(), other_end = gtid_set.intervals.end(); + while (mine != my_end && other != other_end) + { + bool mine_contains_other = mine->start <= other->start && mine->end >= other->end; + if (mine_contains_other) + { + ++other; + } + else + { + ++mine; + } + } + + return other == other_end; //if we've iterated through all intervals in the argument, all its intervals are contained in this +} + +bool GTIDSets::contains(const GTIDSet & gtid_set) const +{ + for (const auto & my_gtid_set : sets) + { + if (my_gtid_set.contains(gtid_set)) { return true; } + } + return false; +} + +bool GTIDSets::contains(const GTIDSets & gtid_sets) const +{ + for (const auto & gtid_set : gtid_sets.sets) + { + if (!this->contains(gtid_set)) { return false; } + } + return true; +} + } diff --git a/src/Core/MySQL/MySQLGtid.h b/src/Core/MySQL/MySQLGtid.h index 45eeaf02fa2..b7cff39cca6 100644 --- a/src/Core/MySQL/MySQLGtid.h +++ b/src/Core/MySQL/MySQLGtid.h @@ -28,6 +28,8 @@ public: void tryMerge(size_t i); static void tryShrink(GTIDSet & set, unsigned int i, Interval & current); + + bool contains(const GTIDSet & gtid_set) const; }; class GTIDSets @@ -40,6 +42,31 @@ public: String toString() const; String toPayload() const; + bool contains(const GTIDSet & gtid_set) const; + bool contains(const GTIDSets & gtid_sets) const; }; +inline bool operator==(const GTID & left, const GTID & right) +{ + return left.uuid == right.uuid + && left.seq_no == right.seq_no; +} + +inline bool operator==(const GTIDSet::Interval & left, const GTIDSet::Interval & right) +{ + return left.start == right.start + && left.end == right.end; +} + +inline bool operator==(const GTIDSet & left, const GTIDSet & right) +{ + return left.uuid == right.uuid + && left.intervals == right.intervals; +} + +inline bool operator==(const GTIDSets & left, const GTIDSets & right) +{ + return left.sets == right.sets; +} + } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index dcf42134b0b..403f98360c1 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -967,6 +967,59 @@ namespace MySQLReplication out << "[DryRun Event]" << '\n'; } + void UnparsedRowsEvent::dump(WriteBuffer & out) const + { + std::lock_guard lock(mutex); + header.dump(out); + out << "[UnparsedRowsEvent Event]" << '\n'; + out << "Unparsed Data Size: " << unparsed_data.size() << '\n'; + } + + void UnparsedRowsEvent::parseImpl(ReadBuffer & payload_) + { + char c = 0; + if (payload_.position() < payload_.buffer().end()) + unparsed_data.reserve(payload_.buffer().end() - payload_.position()); + /// Prevent reading after the end + /// payload.available() might have incorrect value + while (payload_.position() <= payload_.buffer().end() && payload_.read(c)) + unparsed_data.push_back(c); + if (!payload_.eof()) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Available {} bytes but not eof", payload_.available()); + } + + std::shared_ptr UnparsedRowsEvent::parse() + { + std::lock_guard lock(mutex); + if (!unparsed_data.empty()) + { + RowsEventHeader rows_header(header.type); + rows_header.table_id = table_id; + rows_header.flags = flags; + switch (header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown event type: {}", magic_enum::enum_name(header.type)); + } + ReadBufferFromMemory payload(unparsed_data.data(), unparsed_data.size()); + parsed_event->parseEvent(payload); + unparsed_data.clear(); + } + return parsed_event; + } + /// Update binlog name/position/gtid based on the event type. void Position::update(BinlogEventPtr event) { @@ -998,7 +1051,8 @@ namespace MySQLReplication case ROTATE_EVENT: { auto rotate = std::static_pointer_cast(event); binlog_name = rotate->next_binlog; - binlog_pos = event->header.log_pos; + /// If binlog name has changed, need to use position from next binlog + binlog_pos = rotate->position; break; } case GTID_EVENT: { @@ -1012,13 +1066,18 @@ namespace MySQLReplication default: throw ReplicationError(ErrorCodes::LOGICAL_ERROR, "Position update with unsupported event"); } + if (event->header.timestamp > 0) + { + timestamp = event->header.timestamp; + } } - void Position::update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_) + void Position::update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_, UInt32 binlog_time_) { binlog_pos = binlog_pos_; binlog_name = binlog_name_; gtid_sets.parse(gtid_sets_); + timestamp = binlog_time_; } void Position::dump(WriteBuffer & out) const diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 1584dbd42ac..6ba507245b3 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -181,6 +181,7 @@ namespace MySQLReplication MYSQL_WRITE_ROWS_EVENT = 2, MYSQL_UPDATE_ROWS_EVENT = 3, MYSQL_DELETE_ROWS_EVENT = 4, + MYSQL_UNPARSED_ROWS_EVENT = 100, }; class ReplicationError : public DB::Exception @@ -274,6 +275,8 @@ namespace MySQLReplication String status; String schema; String query; + String query_database_name; + String query_table_name; QueryType typ = QUERY_EVENT_DDL; bool transaction_complete = true; @@ -446,7 +449,6 @@ namespace MySQLReplication void parseImpl(ReadBuffer & payload) override; void parseRow(ReadBuffer & payload, Bitmap & bitmap); - private: std::shared_ptr table_map; }; @@ -497,17 +499,38 @@ namespace MySQLReplication void parseImpl(ReadBuffer & payload) override; }; + class UnparsedRowsEvent : public RowsEvent + { + public: + UnparsedRowsEvent(const std::shared_ptr & table_map_, EventHeader && header_, const RowsEventHeader & rows_header) + : RowsEvent(table_map_, std::move(header_), rows_header) + { + } + + void dump(WriteBuffer & out) const override; + MySQLEventType type() const override { return MYSQL_UNPARSED_ROWS_EVENT; } + std::shared_ptr parse(); + + protected: + void parseImpl(ReadBuffer & payload) override; + std::vector unparsed_data; + std::shared_ptr parsed_event; + mutable std::mutex mutex; + }; + class Position { public: UInt64 binlog_pos; String binlog_name; GTIDSets gtid_sets; + UInt32 timestamp; - Position() : binlog_pos(0) { } + Position() : binlog_pos(0), timestamp(0) { } void update(BinlogEventPtr event); - void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_); + void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_, UInt32 binlog_time_); void dump(WriteBuffer & out) const; + void resetPendingGTID() { pending_gtid.reset(); } private: std::optional pending_gtid; diff --git a/src/Core/MySQL/tests/gtest_MySQLGtid.cpp b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp new file mode 100644 index 00000000000..e31a87aaa39 --- /dev/null +++ b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp @@ -0,0 +1,40 @@ +#include +#include + +using namespace DB; + + +GTEST_TEST(GTIDSetsContains, Tests) +{ + GTIDSets gtid_set, + contained1, contained2, contained3, contained4, contained5, + not_contained1, not_contained2, not_contained3, not_contained4, not_contained5, not_contained6; + + gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:2-3:11:47-49"); + contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:11"); + contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:47-49:60"); + contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:60"); + + not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + not_contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:0-3:11:47-49"); + not_contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:99"); + not_contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:46-49:60"); + not_contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:99"); + not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60, 00000000-0000-0000-0000-000000000000"); + + + ASSERT_TRUE(gtid_set.contains(contained1)); + ASSERT_TRUE(gtid_set.contains(contained2)); + ASSERT_TRUE(gtid_set.contains(contained3)); + ASSERT_TRUE(gtid_set.contains(contained4)); + ASSERT_TRUE(gtid_set.contains(contained5)); + + ASSERT_FALSE(gtid_set.contains(not_contained1)); + ASSERT_FALSE(gtid_set.contains(not_contained2)); + ASSERT_FALSE(gtid_set.contains(not_contained3)); + ASSERT_FALSE(gtid_set.contains(not_contained4)); + ASSERT_FALSE(gtid_set.contains(not_contained5)); + ASSERT_FALSE(gtid_set.contains(not_contained6)); +} diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index cbb080a0baa..99dd337189c 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -10,6 +10,7 @@ # include # include # include +# include # include # include # include @@ -39,10 +40,11 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, std::unique_ptr settings_) : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializedMySQL(" + database_name_ + ")", context_) , settings(std::move(settings_)) - , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) + , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get()) { } @@ -197,6 +199,7 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) if (!engine->arguments) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + MySQLReplication::BinlogClientPtr binlog_client; StorageMySQL::Configuration configuration; ASTs & arguments = engine->arguments->children; auto mysql_settings = std::make_unique(); @@ -241,6 +244,12 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) if (engine_define->settings) materialize_mode_settings->loadFromQuery(*engine_define); + if (materialize_mode_settings->use_binlog_client) + binlog_client = DB::MySQLReplication::BinlogClientFactory::instance().getClient( + configuration.host, configuration.port, configuration.username, configuration.password, + materialize_mode_settings->max_bytes_in_binlog_dispatcher_buffer, + materialize_mode_settings->max_flush_milliseconds_in_binlog_dispatcher); + if (args.uuid == UUIDHelpers::Nil) { auto print_create_ast = args.create_query.clone(); @@ -261,6 +270,7 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) configuration.database, std::move(mysql_pool), std::move(client), + binlog_client, std::move(materialize_mode_settings)); }; factory.registerDatabase("MaterializeMySQL", create_fn); diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 895498723fd..4d7871d49d6 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -31,6 +32,7 @@ public: const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, std::unique_ptr settings_); void rethrowExceptionIfNeeded() const; diff --git a/src/Databases/MySQL/MaterializedMySQLSettings.h b/src/Databases/MySQL/MaterializedMySQLSettings.h index 82342f8c76d..557d48be85b 100644 --- a/src/Databases/MySQL/MaterializedMySQLSettings.h +++ b/src/Databases/MySQL/MaterializedMySQLSettings.h @@ -17,6 +17,11 @@ class ASTStorage; M(Int64, max_wait_time_when_mysql_unavailable, 1000, "Retry interval when MySQL is not available (milliseconds). Negative value disable retry.", 0) \ M(Bool, allows_query_when_mysql_lost, false, "Allow query materialized table when mysql is lost.", 0) \ M(String, materialized_mysql_tables_list, "", "a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated.", 0) \ + M(Bool, use_binlog_client, false, "Use MySQL Binlog Client.", 0) \ + M(UInt64, max_bytes_in_binlog_queue, 64 * 1024 * 1024, "Max bytes in binlog's queue created from MySQL Binlog Client.", 0) \ + M(UInt64, max_milliseconds_to_wait_in_binlog_queue, 10000, "Max milliseconds to wait when max bytes exceeded in a binlog queue.", 0) \ + M(UInt64, max_bytes_in_binlog_dispatcher_buffer, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes in the binlog dispatcher's buffer before it is flushed to attached binlogs.", 0) \ + M(UInt64, max_flush_milliseconds_in_binlog_dispatcher, 1000, "Max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlogs.", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 14cd89e1ff6..5834fb96dc6 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -26,14 +26,13 @@ #include #include #include -#include +#include #include #include #include #include #include #include -#include namespace DB { @@ -48,8 +47,43 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_EXCEPTION; extern const int CANNOT_READ_ALL_DATA; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int NETWORK_ERROR; + extern const int UNKNOWN_TABLE; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int THERE_IS_NO_QUERY; + extern const int QUERY_WAS_CANCELLED; + extern const int TABLE_ALREADY_EXISTS; + extern const int DATABASE_ALREADY_EXISTS; + extern const int DATABASE_NOT_EMPTY; + extern const int TABLE_IS_DROPPED; + extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; + extern const int CANNOT_CREATE_CHARSET_CONVERTER; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int TIMEOUT_EXCEEDED; + extern const int MEMORY_LIMIT_EXCEEDED; + extern const int MYSQL_SYNTAX_ERROR; } +// USE MySQL ERROR CODE: +// https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html +constexpr int ER_ACCESS_DENIED_ERROR = 1045; /// NOLINT +constexpr int ER_DBACCESS_DENIED_ERROR = 1044; /// NOLINT +constexpr int ER_BAD_DB_ERROR = 1049; /// NOLINT +constexpr int ER_MASTER_HAS_PURGED_REQUIRED_GTIDS = 1789; /// NOLINT +constexpr int ER_MASTER_FATAL_ERROR_READING_BINLOG = 1236; /// NOLINT + +// https://dev.mysql.com/doc/mysql-errors/8.0/en/client-error-reference.html +constexpr int CR_CONN_HOST_ERROR = 2003; /// NOLINT +constexpr int CR_SERVER_GONE_ERROR = 2006; /// NOLINT +constexpr int CR_SERVER_LOST = 2013; /// NOLINT +constexpr int ER_SERVER_SHUTDOWN = 1053; /// NOLINT +constexpr int ER_LOCK_DEADLOCK = 1213; /// NOLINT +constexpr int ER_LOCK_WAIT_TIMEOUT = 1205; /// NOLINT +constexpr int ER_OPTION_PREVENTS_STATEMENT = 1290; /// NOLINT + static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; static ContextMutablePtr createQueryContext(ContextPtr context) @@ -157,12 +191,68 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S } } +static bool shouldReconnectOnException(const std::exception_ptr & e) +{ + try + { + std::rethrow_exception(e); + } + catch (const mysqlxx::ConnectionFailed &) {} /// NOLINT + catch (const mysqlxx::ConnectionLost &) {} /// NOLINT + catch (const Poco::Net::ConnectionResetException &) {} /// NOLINT + catch (const Poco::Net::ConnectionRefusedException &) {} /// NOLINT + catch (const DB::NetException &) {} /// NOLINT + catch (const Poco::Net::NetException & e) + { + if (e.code() != POCO_ENETDOWN && + e.code() != POCO_ENETUNREACH && + e.code() != POCO_ENETRESET && + e.code() != POCO_ESYSNOTREADY) + return false; + } + catch (const mysqlxx::BadQuery & e) + { + // Lost connection to MySQL server during query + if (e.code() != CR_SERVER_LOST && + e.code() != ER_SERVER_SHUTDOWN && + e.code() != CR_SERVER_GONE_ERROR && + e.code() != CR_CONN_HOST_ERROR && + e.code() != ER_LOCK_DEADLOCK && + e.code() != ER_LOCK_WAIT_TIMEOUT && + e.code() != ER_OPTION_PREVENTS_STATEMENT) + return false; + } + catch (const mysqlxx::Exception & e) + { + // ER_SERVER_SHUTDOWN is thrown in different types under different conditions. + // E.g. checkError() in Common/mysqlxx/Exception.cpp will throw mysqlxx::Exception. + if (e.code() != CR_SERVER_LOST && e.code() != ER_SERVER_SHUTDOWN && e.code() != CR_SERVER_GONE_ERROR && e.code() != CR_CONN_HOST_ERROR) + return false; + } + catch (const Poco::Exception & e) + { + if (e.code() != ErrorCodes::NETWORK_ERROR && + e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED && + e.code() != ErrorCodes::UNKNOWN_TABLE && // Since we have ignored the DDL exception when the tables without primary key, insert into those tables will get UNKNOWN_TABLE. + e.code() != ErrorCodes::CANNOT_READ_ALL_DATA && + e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::TIMEOUT_EXCEEDED) + return false; + } + catch (...) + { + return false; + } + return true; +} + MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( ContextPtr context_, const String & database_name_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, MaterializedMySQLSettings * settings_) : WithContext(context_->getGlobalContext()) , log(&Poco::Logger::get("MaterializedMySQLSyncThread")) @@ -170,6 +260,7 @@ MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( , mysql_database_name(mysql_database_name_) , pool(std::move(pool_)) /// NOLINT , client(std::move(client_)) + , binlog_client(binlog_client_) , settings(settings_) { query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; @@ -216,14 +307,23 @@ void MaterializedMySQLSyncThread::synchronization() UInt64 elapsed_ms = watch.elapsedMilliseconds(); if (elapsed_ms < max_flush_time) { - BinlogEventPtr binlog_event = client.readOneBinlogEvent(max_flush_time - elapsed_ms); - if (binlog_event) + const auto timeout_ms = max_flush_time - elapsed_ms; + BinlogEventPtr binlog_event; + if (binlog) + binlog->tryReadEvent(binlog_event, timeout_ms); + else + binlog_event = client.readOneBinlogEvent(timeout_ms); + if (binlog_event && !ignoreEvent(binlog_event)) onEvent(buffers, binlog_event, metadata); } } catch (const Exception & e) { - if (e.code() != ErrorCodes::CANNOT_READ_ALL_DATA || settings->max_wait_time_when_mysql_unavailable < 0) + if (settings->max_wait_time_when_mysql_unavailable < 0) + throw; + bool binlog_was_purged = e.code() == ER_MASTER_FATAL_ERROR_READING_BINLOG || + e.code() == ER_MASTER_HAS_PURGED_REQUIRED_GTIDS; + if (!binlog_was_purged && !shouldReconnectOnException(std::current_exception())) throw; flushBuffersData(buffers, metadata); @@ -246,6 +346,7 @@ void MaterializedMySQLSyncThread::synchronization() catch (...) { client.disconnect(); + binlog = nullptr; tryLogCurrentException(log); setSynchronizationThreadException(std::current_exception()); } @@ -259,6 +360,7 @@ void MaterializedMySQLSyncThread::stopSynchronization() if (background_thread_pool->joinable()) background_thread_pool->join(); client.disconnect(); + binlog = nullptr; } } @@ -428,14 +530,6 @@ static inline void dumpDataForTables( } } -static inline UInt32 randomNumber() -{ - pcg64_fast rng{randomSeed()}; - std::uniform_int_distribution dist6( - std::numeric_limits::min(), std::numeric_limits::max()); - return static_cast(dist6(rng)); -} - bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metadata) { bool opened_transaction = false; @@ -463,7 +557,7 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (!need_dumping_tables.empty()) { Position position; - position.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set); + position.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set, 0); metadata.transaction(position, [&]() { @@ -487,8 +581,20 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (opened_transaction) connection->query("COMMIT").execute(); - client.connect(); - client.startBinlogDumpGTID(randomNumber(), mysql_database_name, materialized_tables_list, metadata.executed_gtid_set, metadata.binlog_checksum); + if (binlog_client) + { + binlog_client->setBinlogChecksum(metadata.binlog_checksum); + binlog = binlog_client->createBinlog(metadata.executed_gtid_set, + database_name, + {mysql_database_name}, + settings->max_bytes_in_binlog_queue, + settings->max_milliseconds_to_wait_in_binlog_queue); + } + else + { + client.connect(); + client.startBinlogDumpGTID(randomNumber(), mysql_database_name, materialized_tables_list, metadata.executed_gtid_set, metadata.binlog_checksum); + } setSynchronizationThreadException(nullptr); return true; @@ -500,20 +606,11 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (opened_transaction) connection->query("ROLLBACK").execute(); - try - { + if (settings->max_wait_time_when_mysql_unavailable < 0) + throw; + + if (!shouldReconnectOnException(std::current_exception())) throw; - } - catch (const mysqlxx::ConnectionFailed & ex) - { - LOG_TRACE(log, "Connection to MySQL failed {}", ex.displayText()); - } - catch (const mysqlxx::BadQuery & e) - { - // Lost connection to MySQL server during query - if (e.code() != CR_SERVER_LOST || settings->max_wait_time_when_mysql_unavailable < 0) - throw; - } setSynchronizationThreadException(std::current_exception()); /// Avoid busy loop when MySQL is not available. @@ -524,17 +621,55 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta return false; } +bool MaterializedMySQLSyncThread::isTableIgnored(const String & table_name) const +{ + return !materialized_tables_list.empty() && !materialized_tables_list.contains(table_name); +} + +bool MaterializedMySQLSyncThread::ignoreEvent(const BinlogEventPtr & event) const +{ + switch (event->type()) + { + case MYSQL_WRITE_ROWS_EVENT: + case MYSQL_DELETE_ROWS_EVENT: + case MYSQL_UPDATE_ROWS_EVENT: + case MYSQL_UNPARSED_ROWS_EVENT: + { + auto table_name = static_cast(*event).table; + if (!table_name.empty() && isTableIgnored(table_name)) + { + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown event type: {}", magic_enum::enum_name(event->header.type)); + } + return true; + } + } break; + default: + break; + } + return false; +} + void MaterializedMySQLSyncThread::flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata) { if (buffers.data.empty()) return; - metadata.transaction(client.getPosition(), [&]() { buffers.commit(getContext()); }); + metadata.transaction(getPosition(), [&]() { buffers.commit(getContext()); }); const auto & position_message = [&]() { WriteBufferFromOwnString buf; - client.getPosition().dump(buf); + getPosition().dump(buf); return buf.str(); }; LOG_INFO(log, "MySQL executed position: \n {}", position_message()); @@ -783,10 +918,33 @@ void MaterializedMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPt else if (receive_event->type() == MYSQL_QUERY_EVENT) { QueryEvent & query_event = static_cast(*receive_event); + /// Skip events for different databases if any + if (!query_event.query_database_name.empty() && query_event.query_database_name != mysql_database_name) + { + LOG_WARNING( + log, + "Skipped QueryEvent, current mysql database name: {}, ddl schema: {}, query: {}", + mysql_database_name, + query_event.query_database_name, + query_event.query); + return; + } + if (!query_event.query_table_name.empty() && isTableIgnored(query_event.query_table_name)) + { + LOG_WARNING(log, "Due to the table filter rules, query_event on {} is ignored.", database_name); + return; + } + Position position_before_ddl; - position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set); + position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set, query_event.header.timestamp); metadata.transaction(position_before_ddl, [&]() { buffers.commit(getContext()); }); - metadata.transaction(client.getPosition(),[&](){ executeDDLAtomic(query_event); }); + metadata.transaction(getPosition(),[&]() { executeDDLAtomic(query_event); }); + } + else if (receive_event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + UnparsedRowsEvent & unparsed_event = static_cast(*receive_event); + auto nested_event = unparsed_event.parse(); + onEvent(buffers, nested_event, metadata); } else { @@ -796,7 +954,10 @@ void MaterializedMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPt /// Some behaviors(such as changing the value of "binlog_checksum") rotate the binlog file. /// To ensure that the synchronization continues, we need to handle these events metadata.fetchMasterVariablesValue(pool.get(/* wait_timeout= */ UINT64_MAX)); - client.setBinlogChecksum(metadata.binlog_checksum); + if (binlog_client) + binlog_client->setBinlogChecksum(metadata.binlog_checksum); + else + client.setBinlogChecksum(metadata.binlog_checksum); } else if (receive_event->header.type != HEARTBEAT_EVENT) { @@ -827,7 +988,7 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even auto table_id = tryParseTableIDFromDDL(query, query_event.schema); if (!table_id.table_name.empty()) { - if (table_id.database_name != mysql_database_name || !materialized_tables_list.contains(table_id.table_name)) + if (table_id.database_name != mysql_database_name || isTableIgnored(table_id.table_name)) { LOG_DEBUG(log, "Skip MySQL DDL for {}.{}:\n{}", table_id.database_name, table_id.table_name, query); return; @@ -845,8 +1006,28 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even tryLogCurrentException(log); /// If some DDL query was not successfully parsed and executed - /// Then replication may fail on next binlog events anyway - if (exception.code() != ErrorCodes::SYNTAX_ERROR) + /// Then replication may fail on next binlog events anyway. + /// We can skip the error binlog evetns and continue to execute the right ones. + /// eg. The user creates a table without primary key and finds it is wrong, then + /// drops it and creates a new right one. We guarantee the right one can be executed. + + if (exception.code() != ErrorCodes::SYNTAX_ERROR && + exception.code() != ErrorCodes::MYSQL_SYNTAX_ERROR && + exception.code() != ErrorCodes::NOT_IMPLEMENTED && + exception.code() != ErrorCodes::UNKNOWN_TABLE && + exception.code() != ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY && + exception.code() != ErrorCodes::THERE_IS_NO_QUERY && + exception.code() != ErrorCodes::QUERY_WAS_CANCELLED && + exception.code() != ErrorCodes::TABLE_ALREADY_EXISTS && + exception.code() != ErrorCodes::UNKNOWN_DATABASE && + exception.code() != ErrorCodes::DATABASE_ALREADY_EXISTS && + exception.code() != ErrorCodes::DATABASE_NOT_EMPTY && + exception.code() != ErrorCodes::TABLE_IS_DROPPED && + exception.code() != ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT && + exception.code() != ErrorCodes::CANNOT_CREATE_CHARSET_CONVERTER && + exception.code() != ErrorCodes::UNKNOWN_FUNCTION && + exception.code() != ErrorCodes::UNKNOWN_IDENTIFIER && + exception.code() != ErrorCodes::UNKNOWN_TYPE) throw; } } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 4abea5e72df..004a4d67d32 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -11,6 +11,7 @@ # include # include # include +# include # include # include # include @@ -45,6 +46,7 @@ public: const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, MaterializedMySQLSettings * settings_); void stopSynchronization(); @@ -61,19 +63,12 @@ private: mutable mysqlxx::Pool pool; mutable MySQLClient client; + BinlogClientPtr binlog_client; + BinlogPtr binlog; MaterializedMySQLSettings * settings; String query_prefix; NameSet materialized_tables_list; - // USE MySQL ERROR CODE: - // https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html - const int ER_ACCESS_DENIED_ERROR = 1045; /// NOLINT - const int ER_DBACCESS_DENIED_ERROR = 1044; /// NOLINT - const int ER_BAD_DB_ERROR = 1049; /// NOLINT - - // https://dev.mysql.com/doc/mysql-errors/8.0/en/client-error-reference.html - const int CR_SERVER_LOST = 2013; /// NOLINT - struct Buffers { String database; @@ -99,12 +94,16 @@ private: BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, ContextPtr context); }; + Position getPosition() const { return binlog ? binlog->getPosition() : client.getPosition(); } void synchronization(); bool isCancelled() { return sync_quit.load(std::memory_order_relaxed); } bool prepareSynchronized(MaterializeMetadata & metadata); + bool isTableIgnored(const String & table_name) const; + bool ignoreEvent(const BinlogEventPtr & event) const; + void flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata); void onEvent(Buffers & buffers, const MySQLReplication::BinlogEventPtr & event, MaterializeMetadata & metadata); diff --git a/src/Databases/MySQL/MySQLBinlog.cpp b/src/Databases/MySQL/MySQLBinlog.cpp new file mode 100644 index 00000000000..3e3aca220bb --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlog.cpp @@ -0,0 +1,500 @@ +#include "MySQLBinlog.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +using namespace Replication; +using namespace Authentication; +using namespace ConnectionPhase; + +namespace ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; + extern const int UNKNOWN_PACKET_FROM_SERVER; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int CANNOT_READ_ALL_DATA; + extern const int LOGICAL_ERROR; + extern const int NETWORK_ERROR; +} + +namespace MySQLReplication +{ + +class WriteCommand : public IMySQLWritePacket +{ +public: + const char command; + const String query; + + WriteCommand(char command_, const String & query_) : command(command_), query(query_) { } + + size_t getPayloadSize() const override { return 1 + query.size(); } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(command); + if (!query.empty()) + buffer.write(query.data(), query.size()); + } +}; + +IBinlog::Checksum IBinlog::checksumFromString(const String & checksum) +{ + auto str = Poco::toUpper(checksum); + if (str == "CRC32") + return IBinlog::CRC32; + if (str != "NONE") + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown checksum: {}", checksum); + return IBinlog::NONE; +} + +void BinlogParser::setChecksum(Checksum checksum) +{ + switch (checksum) + { + case Checksum::CRC32: + checksum_signature_length = 4; + break; + case Checksum::NONE: + checksum_signature_length = 0; + break; + } +} + +void BinlogParser::parseEvent(EventHeader & event_header, ReadBuffer & event_payload) +{ + switch (event_header.type) + { + case FORMAT_DESCRIPTION_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case ROTATE_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case QUERY_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + + auto query = std::static_pointer_cast(event); + switch (query->typ) + { + case QUERY_EVENT_MULTI_TXN_FLAG: + case QUERY_EVENT_XA: + case QUERY_SAVEPOINT: + { + event = std::make_shared(EventHeader(query->header)); + break; + } + default: + { + String quoted_query = query->query; + tryQuoteUnrecognizedTokens(quoted_query); + tryConvertStringLiterals(quoted_query); + auto table_id = tryParseTableIDFromDDL(query->query, query->schema); + query->query_database_name = table_id.database_name; + query->query_table_name = table_id.table_name; + break; + } + } + break; + } + case XID_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case TABLE_MAP_EVENT: + { + TableMapEventHeader map_event_header; + map_event_header.parse(event_payload); + event = std::make_shared(EventHeader(event_header), map_event_header, flavor_charset); + try + { + event->parseEvent(event_payload); + auto table_map = std::static_pointer_cast(event); + table_maps[table_map->table_id] = table_map; + } + catch (const Poco::Exception & exc) + { + /// Ignore parsing issues + if (exc.code() != ErrorCodes::UNKNOWN_EXCEPTION) + throw; + event = std::make_shared(std::move(event_header)); + event->parseEvent(event_payload); + } + break; + } + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + { + RowsEventHeader rows_header(event_header.type); + rows_header.parse(event_payload); + if (table_maps.contains(rows_header.table_id)) + event = std::make_shared(table_maps.at(rows_header.table_id), EventHeader(event_header), rows_header); + else + event = std::make_shared(std::move(event_header)); + event->parseEvent(event_payload); + if (rows_header.flags & ROWS_END_OF_STATEMENT) + table_maps.clear(); + break; + } + case GTID_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + default: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + } + updatePosition(event, position); +} + +void BinlogParser::updatePosition(const BinlogEventPtr & event, Position & position) +{ + const UInt64 binlog_pos_prev = position.binlog_pos; + position.binlog_pos = event->header.log_pos; + if (event->header.timestamp > 0) + position.timestamp = event->header.timestamp; + + switch (event->header.type) + { + case QUERY_EVENT: + if (event->type() == MYSQL_UNHANDLED_EVENT) + break; + [[fallthrough]]; + case GTID_EVENT: + case XID_EVENT: + case ROTATE_EVENT: + position.update(event); + break; + default: + break; + } + + if (event->header.type != ROTATE_EVENT) + { + /// UInt32 overflow when Pos > End_log_pos + /// https://dev.mysql.com/doc/refman/8.0/en/show-binlog-events.html + /// binlog_pos - The position at which the next event begins, which is equal to Pos plus the size of the event + const UInt64 binlog_pos_correct = binlog_pos_prev + event->header.event_size; + if (position.binlog_pos < binlog_pos_prev && binlog_pos_correct > std::numeric_limits::max()) + position.binlog_pos = binlog_pos_correct; + } +} + +bool BinlogParser::isNew(const Position & older, const Position & newer) +{ + if (older.gtid_sets.contains(newer.gtid_sets)) + return false; + /// Check if all sets in newer position have the same UUID from older sets + std::set older_set; + for (const auto & set : older.gtid_sets.sets) + older_set.insert(set.uuid); + for (const auto & set : newer.gtid_sets.sets) + { + if (!older_set.contains(set.uuid)) + return false; + } + return true; +} + +void BinlogFromSocket::connect(const String & host, UInt16 port, const String & user, const String & password) +{ + if (connected) + disconnect(); + + const Poco::Timespan connection_timeout(10'000'000'000); + const Poco::Timespan receive_timeout(5'000'000'000); + const Poco::Timespan send_timeout(5'000'000'000); + + socket = std::make_unique(); + address = DNSResolver::instance().resolveAddress(host, port); + socket->connect(*address, connection_timeout); + socket->setReceiveTimeout(receive_timeout); + socket->setSendTimeout(send_timeout); + socket->setNoDelay(true); + connected = true; + + in = std::make_unique(*socket); + out = std::make_unique(*socket); + packet_endpoint = std::make_shared(*in, *out, sequence_id); + + handshake(user, password); +} + +void BinlogFromSocket::disconnect() +{ + in = nullptr; + out = nullptr; + if (socket) + socket->close(); + socket = nullptr; + connected = false; + sequence_id = 0; + + GTIDSets sets; + position.gtid_sets = sets; + position.resetPendingGTID(); +} + +/// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html +void BinlogFromSocket::handshake(const String & user, const String & password) +{ + const String mysql_native_password = "mysql_native_password"; + Handshake handshake; + packet_endpoint->receivePacket(handshake); + if (handshake.auth_plugin_name != mysql_native_password) + { + throw Exception( + ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, + "Only support {} auth plugin name, but got {}", + mysql_native_password, + handshake.auth_plugin_name); + } + + Native41 native41(password, handshake.auth_plugin_data); + String auth_plugin_data = native41.getAuthPluginData(); + + const UInt8 charset_utf8 = 33; + HandshakeResponse handshake_response( + client_capabilities, MAX_PACKET_LENGTH, charset_utf8, user, "", auth_plugin_data, mysql_native_password); + packet_endpoint->sendPacket(handshake_response, true); + + ResponsePacket packet_response(client_capabilities, true); + packet_endpoint->receivePacket(packet_response); + packet_endpoint->resetSequenceId(); + + if (packet_response.getType() == PACKET_ERR) + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + else if (packet_response.getType() == PACKET_AUTH_SWITCH) + throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Access denied for user {}", user); +} + +void BinlogFromSocket::writeCommand(char command, const String & query) +{ + WriteCommand write_command(command, query); + packet_endpoint->sendPacket(write_command, true); + + ResponsePacket packet_response(client_capabilities); + packet_endpoint->receivePacket(packet_response); + switch (packet_response.getType()) + { + case PACKET_ERR: + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + case PACKET_OK: + break; + default: + break; + } + packet_endpoint->resetSequenceId(); +} + +void BinlogFromSocket::registerSlaveOnMaster(UInt32 slave_id) +{ + RegisterSlave register_slave(slave_id); + packet_endpoint->sendPacket(register_slave, true); + + ResponsePacket packet_response(client_capabilities); + packet_endpoint->receivePacket(packet_response); + packet_endpoint->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); +} + +void BinlogFromSocket::start(UInt32 slave_id, const String & executed_gtid_set) +{ + if (!connected) + return; + + /// Maybe CRC32 or NONE. mysqlbinlog.cc use NONE, see its below comments: + /// Make a notice to the server that this client is checksum-aware. + /// It does not need the first fake Rotate necessary checksummed. + writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = 'CRC32'"); + + /// Set heartbeat 1s + const UInt64 period_ns = 1'000'000'000; + writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); + + /// Register slave. + registerSlaveOnMaster(slave_id); + + position.gtid_sets = {}; + position.gtid_sets.parse(executed_gtid_set); + + BinlogDumpGTID binlog_dump(slave_id, position.gtid_sets.toPayload()); + packet_endpoint->sendPacket(binlog_dump, true); +} + +class ReadPacketFromSocket : public IMySQLReadPacket +{ +public: + using ReadPayloadFunc = std::function; + explicit ReadPacketFromSocket(ReadPayloadFunc fn) : read_payload_func(std::move(fn)) { } + void readPayloadImpl(ReadBuffer & payload) override; + ReadPayloadFunc read_payload_func; +}; + +void ReadPacketFromSocket::readPayloadImpl(ReadBuffer & payload) +{ + if (payload.eof()) + throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF."); + + UInt8 header = static_cast(*payload.position()); + switch (header) // NOLINT(bugprone-switch-missing-default-case) + { + case PACKET_EOF: + throw ReplicationError(ErrorCodes::CANNOT_READ_ALL_DATA, "Master maybe lost"); + case PACKET_ERR: + { + ERRPacket err; + err.readPayloadWithUnpacked(payload); + throw ReplicationError::createDeprecated(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + } + default: + break; + } + /// Skip the generic response packets header flag + payload.ignore(1); + read_payload_func(payload); +} + +bool BinlogFromSocket::tryReadEvent(BinlogEventPtr & to, UInt64 ms) +{ + ReadPacketFromSocket packet([this](ReadBuffer & payload) + { + MySQLBinlogEventReadBuffer event_payload(payload, checksum_signature_length); + + EventHeader event_header; + event_header.parse(event_payload); + + parseEvent(event_header, event_payload); + }); + + if (packet_endpoint && packet_endpoint->tryReceivePacket(packet, ms)) + { + to = event; + return static_cast(to); + } + + return false; +} + +void BinlogFromFile::open(const String & filename) +{ + in = std::make_unique(filename); + assertString("\xfe\x62\x69\x6e", *in); /// magic number +} + +bool BinlogFromFile::tryReadEvent(BinlogEventPtr & to, UInt64 /*ms*/) +{ + if (in && !in->eof()) + { + EventHeader event_header; + event_header.parse(*in); + + LimitReadBuffer limit_read_buffer(*in, event_header.event_size - EVENT_HEADER_LENGTH, /* throw_exception */ false, /* exact_limit */ {}); + MySQLBinlogEventReadBuffer event_payload(limit_read_buffer, checksum_signature_length); + parseEvent(event_header, event_payload); + to = event; + return static_cast(to); + } + + return false; +} + +BinlogFromFileFactory::BinlogFromFileFactory(const String & filename_) + : filename(filename_) +{ +} + +BinlogPtr BinlogFromFileFactory::createBinlog(const String & executed_gtid_set) +{ + auto ret = std::make_shared(); + ret->open(filename); + if (!executed_gtid_set.empty()) + { + /// NOTE: Used for testing only! + GTIDSets sets; + sets.parse(executed_gtid_set); + if (sets.sets.size() != 1 || sets.sets[0].intervals.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too many intervals: {}", executed_gtid_set); + BinlogEventPtr event; + while (ret->tryReadEvent(event, 0)) + { + const auto & s = ret->getPosition().gtid_sets.sets; + if (!s.empty() && !s[0].intervals.empty() && s[0].intervals[0].end == sets.sets[0].intervals[0].end) + break; + } + + auto pos = ret->getPosition(); + pos.gtid_sets.sets.front().intervals.front().start = sets.sets.front().intervals.front().start; + ret->setPosition(pos); + } + return ret; +} + +BinlogFromSocketFactory::BinlogFromSocketFactory(const String & host_, UInt16 port_, const String & user_, const String & password_) + : host(host_) + , port(port_) + , user(user_) + , password(password_) +{ +} + +BinlogPtr BinlogFromSocketFactory::createBinlog(const String & executed_gtid_set) +{ + auto ret = std::make_shared(); + ret->connect(host, port, user, password); + ret->start(randomNumber(), executed_gtid_set); + auto pos = ret->getPosition(); + if (pos.gtid_sets.sets.empty() || pos.gtid_sets.sets.front().intervals.front().start != 1) + throw Exception(ErrorCodes::NETWORK_ERROR, "Could not create: Wrong executed_gtid_set: {} -> {}", executed_gtid_set, pos.gtid_sets.toString()); + return ret; +} + +/// Should be in MySQLReplication namespace +bool operator==(const Position & left, const Position & right) +{ + return left.binlog_name == right.binlog_name && + left.binlog_pos == right.binlog_pos && + left.gtid_sets == right.gtid_sets; +} + +} +} diff --git a/src/Databases/MySQL/MySQLBinlog.h b/src/Databases/MySQL/MySQLBinlog.h new file mode 100644 index 00000000000..0b8f7543590 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlog.h @@ -0,0 +1,120 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +using namespace MySQLProtocol; +using namespace Generic; + +namespace MySQLReplication +{ + +class IBinlog +{ +public: + virtual ~IBinlog() = default; + virtual bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) = 0; + virtual Position getPosition() const = 0; + enum Checksum : UInt8 + { + NONE = 0, + CRC32 = 1 + }; + virtual void setChecksum(Checksum /*checksum*/) { } + static Checksum checksumFromString(const String & checksum); +}; + +using BinlogPtr = std::shared_ptr; + +class BinlogParser : public IBinlog +{ +public: + Position getPosition() const override { return position; } + void setPosition(const Position & position_) { position = position_; } + void setChecksum(Checksum checksum) override; + static void updatePosition(const BinlogEventPtr & event, Position & position); + /// Checks if \a older is older position than \a newer + static bool isNew(const Position & older, const Position & newer); + +protected: + Position position; + BinlogEventPtr event; + std::map> table_maps; + size_t checksum_signature_length = 4; + MySQLCharsetPtr flavor_charset = std::make_shared(); + void parseEvent(EventHeader & event_header, ReadBuffer & event_payload); +}; + +class BinlogFromSocket : public BinlogParser +{ +public: + void connect(const String & host, UInt16 port, const String & user, const String & password); + void start(UInt32 slave_id, const String & executed_gtid_set); + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + +private: + void disconnect(); + bool connected = false; + uint8_t sequence_id = 0; + const uint32_t client_capabilities = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; + + std::unique_ptr in; + std::unique_ptr out; + std::unique_ptr socket; + std::optional address; + std::shared_ptr packet_endpoint; + + void handshake(const String & user, const String & password); + void registerSlaveOnMaster(UInt32 slave_id); + void writeCommand(char command, const String & query); +}; + +class BinlogFromFile : public BinlogParser +{ +public: + void open(const String & filename); + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + +private: + std::unique_ptr in; +}; + +class IBinlogFactory +{ +public: + virtual ~IBinlogFactory() = default; + virtual BinlogPtr createBinlog(const String & executed_gtid_set) = 0; +}; + +using BinlogFactoryPtr = std::shared_ptr; + +class BinlogFromFileFactory : public IBinlogFactory +{ +public: + BinlogFromFileFactory(const String & filename_); + BinlogPtr createBinlog(const String & executed_gtid_set) override; + +private: + const String filename; +}; + +class BinlogFromSocketFactory : public IBinlogFactory +{ +public: + BinlogFromSocketFactory(const String & host_, UInt16 port_, const String & user_, const String & password_); + BinlogPtr createBinlog(const String & executed_gtid_set) override; + +private: + const String host; + const UInt16 port; + const String user; + const String password; +}; + +bool operator==(const Position & left, const Position & right); + +} +} diff --git a/src/Databases/MySQL/MySQLBinlogClient.cpp b/src/Databases/MySQL/MySQLBinlogClient.cpp new file mode 100644 index 00000000000..e7d707f76ce --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClient.cpp @@ -0,0 +1,104 @@ +#include "MySQLBinlogClient.h" +#include + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB::MySQLReplication +{ + +BinlogClient::BinlogClient(const BinlogFactoryPtr & factory_, + const String & name, + UInt64 max_bytes_in_buffer_, + UInt64 max_flush_ms_) + : factory(factory_) + , binlog_client_name(name) + , max_bytes_in_buffer(max_bytes_in_buffer_) + , max_flush_ms(max_flush_ms_) + , logger(&Poco::Logger::get("BinlogClient(" + name + ")")) +{ +} + +BinlogPtr BinlogClient::createBinlog(const String & executed_gtid_set, + const String & name, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + std::lock_guard lock(mutex); + BinlogPtr ret; + for (auto it = dispatchers.begin(); it != dispatchers.end();) + { + auto & dispatcher = *it; + if (!ret) + { + const auto metadata = dispatcher->getDispatcherMetadata(); + LOG_DEBUG(logger, "({} -> {}): Trying dispatcher: {}, size: {} -> {}:{}.{}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), + metadata.position.binlog_name, metadata.position.gtid_sets.toString(), metadata.position.binlog_pos); + ret = dispatcher->attach(executed_gtid_set, name, mysql_database_names, max_bytes, max_waiting_ms); + if (ret) + LOG_DEBUG(logger, "({} -> {}): Reused dispatcher: {}, size: {} -> {}:{}.{}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), + metadata.position.binlog_name, metadata.position.gtid_sets.toString(), metadata.position.binlog_pos); + } + + if (dispatcher->cleanupBinlogsAndStop()) + { + const auto metadata = dispatcher->getDispatcherMetadata(); + LOG_DEBUG(logger, "({} -> {}): Deleting dispatcher: {}, size: {}, total dispatchers: {}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), dispatchers.size()); + it = dispatchers.erase(it); + continue; + } + ++it; + } + + if (!ret) + { + String dispatcher_name = name + ":" + std::to_string(dispatchers_count++); + LOG_DEBUG(logger, "({} -> {}): Creating dispatcher: {}, total dispatchers: {}", + name, executed_gtid_set, dispatcher_name, dispatchers.size()); + auto dispatcher = std::make_shared(dispatcher_name, max_bytes_in_buffer, max_flush_ms); + if (!binlog_checksum.empty()) + dispatcher->setBinlogChecksum(binlog_checksum); + for (const auto & it : dispatchers) + dispatcher->syncTo(it); + ret = dispatcher->start(factory->createBinlog(executed_gtid_set), name, mysql_database_names, max_bytes, max_waiting_ms); + if (!ret) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not create binlog: {}", executed_gtid_set); + dispatchers.push_back(dispatcher); + } + + return ret; +} + +BinlogClient::Metadata BinlogClient::getMetadata() const +{ + std::lock_guard lock(mutex); + Metadata ret; + ret.binlog_client_name = binlog_client_name; + for (const auto & dispatcher : dispatchers) + { + auto metadata = dispatcher->getDispatcherMetadata(); + if (!metadata.binlogs.empty()) + ret.dispatchers.push_back(metadata); + } + return ret; +} + +void BinlogClient::setBinlogChecksum(const String & checksum) +{ + std::lock_guard lock(mutex); + if (binlog_checksum != checksum) + { + LOG_DEBUG(logger, "Setting binlog_checksum: {} -> {}, total dispatchers: {}", binlog_checksum, checksum, dispatchers.size()); + binlog_checksum = checksum; + for (const auto & dispatcher : dispatchers) + dispatcher->setBinlogChecksum(checksum); + } +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogClient.h b/src/Databases/MySQL/MySQLBinlogClient.h new file mode 100644 index 00000000000..b76934d08cf --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClient.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + +namespace DB::MySQLReplication +{ + +/** It is supposed to reduce the number of connections to remote MySQL binlog by reusing one connection between several consumers. + * Such reusing of the connection makes the time of reading from the remote binlog independent to number of the consumers. + * It tracks a list of BinlogEventsDispatcher instances for consumers with different binlog position. + * The dispatchers with the same binlog position will be merged to one. + */ +class BinlogClient +{ +public: + BinlogClient(const BinlogFactoryPtr & factory, + const String & name = {}, + UInt64 max_bytes_in_buffer_ = DBMS_DEFAULT_BUFFER_SIZE, + UInt64 max_flush_ms_ = 1000); + BinlogClient(const BinlogClient & other) = delete; + ~BinlogClient() = default; + BinlogClient & operator=(const BinlogClient & other) = delete; + + /// Creates a binlog to receive events + BinlogPtr createBinlog(const String & executed_gtid_set = {}, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /// The binlog checksum is related to entire connection + void setBinlogChecksum(const String & checksum); + + struct Metadata + { + String binlog_client_name; + std::vector dispatchers; + }; + /// Returns only not empty dispatchers + Metadata getMetadata() const; + +private: + BinlogFactoryPtr factory; + const String binlog_client_name; + UInt64 max_bytes_in_buffer = 0; + UInt64 max_flush_ms = 0; + std::vector dispatchers; + String binlog_checksum; + mutable std::mutex mutex; + Poco::Logger * logger = nullptr; + int dispatchers_count = 0; +}; + +using BinlogClientPtr = std::shared_ptr; + +} diff --git a/src/Databases/MySQL/MySQLBinlogClientFactory.cpp b/src/Databases/MySQL/MySQLBinlogClientFactory.cpp new file mode 100644 index 00000000000..03a777ff352 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClientFactory.cpp @@ -0,0 +1,46 @@ +#include + +namespace DB::MySQLReplication +{ + +BinlogClientFactory & BinlogClientFactory::instance() +{ + static BinlogClientFactory ret; + return ret; +} + +BinlogClientPtr BinlogClientFactory::getClient(const String & host, UInt16 port, const String & user, const String & password, UInt64 max_bytes_in_buffer, UInt64 max_flush_ms) +{ + std::lock_guard lock(mutex); + String binlog_client_name; + WriteBufferFromString stream(binlog_client_name); + stream << user << "@" << host << ":" << port; + stream.finalize(); + String binlog_client_key = binlog_client_name + ":" + password; + auto it = clients.find(binlog_client_key); + BinlogClientPtr ret = it != clients.end() ? it->second.lock() : nullptr; + if (ret) + return ret; + auto factory = std::make_shared(host, port, user, password); + auto client = std::make_shared(factory, binlog_client_name, max_bytes_in_buffer, max_flush_ms); + clients[binlog_client_key] = client; + return client; +} + +std::vector BinlogClientFactory::getMetadata() const +{ + std::lock_guard lock(mutex); + std::vector ret; + for (const auto & it : clients) + { + if (auto c = it.second.lock()) + { + auto metadata = c->getMetadata(); + if (!metadata.dispatchers.empty()) + ret.push_back(metadata); + } + } + return ret; +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogClientFactory.h b/src/Databases/MySQL/MySQLBinlogClientFactory.h new file mode 100644 index 00000000000..544b88e3201 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClientFactory.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + +namespace DB::MySQLReplication +{ + +/** Global instance to create or reuse MySQL Binlog Clients. + * If a binlog client already exists for specific params, + * it will be returned and reused to read binlog events from MySQL. + * Otherwise new instance will be created. + */ +class BinlogClientFactory final : boost::noncopyable +{ +public: + static BinlogClientFactory & instance(); + + BinlogClientPtr getClient(const String & host, + UInt16 port, + const String & user, + const String & password, + UInt64 max_bytes_in_buffer, + UInt64 max_flush_ms); + + /// Returns info of all registered clients + std::vector getMetadata() const; + +private: + BinlogClientFactory() = default; + + // Keeps track of already destroyed clients + std::unordered_map> clients; + mutable std::mutex mutex; +}; + +} diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp new file mode 100644 index 00000000000..4af307f9c0f --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp @@ -0,0 +1,626 @@ +#include "MySQLBinlogEventsDispatcher.h" +#include +#include + +namespace DB::ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; + extern const int TIMEOUT_EXCEEDED; +} + +namespace DB::MySQLReplication +{ + +class BinlogFromDispatcher : public IBinlog +{ +public: + BinlogFromDispatcher(const String & name_, const NameSet & mysql_database_names_, size_t max_bytes_, UInt64 max_waiting_ms_) + : name(name_) + , mysql_database_names(mysql_database_names_) + , max_bytes(max_bytes_) + , max_waiting_ms(max_waiting_ms_) + , logger(&Poco::Logger::get("BinlogFromDispatcher(" + name + ")")) + { + } + + ~BinlogFromDispatcher() override + { + stop(); + } + + void stop() + { + { + std::lock_guard lock(mutex); + if (is_cancelled) + return; + is_cancelled = true; + } + cv.notify_all(); + } + + std::string getName() const + { + return name; + } + + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + Position getPosition() const override; + void setPosition(const Position & initial, const Position & wait); + void setException(const std::exception_ptr & exception_); + void push(const BinlogEventsDispatcher::Buffer & buffer); + BinlogEventsDispatcher::BinlogMetadata getBinlogMetadata() const; + +private: + const String name; + const NameSet mysql_database_names; + const size_t max_bytes = 0; + const UInt64 max_waiting_ms = 0; + + Position position; + GTIDSets gtid_sets_wait; + + BinlogEventsDispatcher::Buffer buffer; + mutable std::mutex mutex; + + std::condition_variable cv; + bool is_cancelled = false; + Poco::Logger * logger = nullptr; + std::exception_ptr exception; +}; + +static String getBinlogNames(const std::vector> & binlogs) +{ + std::vector names; + for (const auto & it : binlogs) + { + if (auto binlog = it.lock()) + names.push_back(binlog->getName()); + } + return boost::algorithm::join(names, ", "); +} + +BinlogEventsDispatcher::BinlogEventsDispatcher(const String & logger_name_, size_t max_bytes_in_buffer_, UInt64 max_flush_ms_) + : logger_name(logger_name_) + , max_bytes_in_buffer(max_bytes_in_buffer_) + , max_flush_ms(max_flush_ms_) + , logger(&Poco::Logger::get("BinlogEventsDispatcher(" + logger_name + ")")) + , dispatching_thread(std::make_unique([this]() { dispatchEvents(); })) +{ +} + +BinlogEventsDispatcher::~BinlogEventsDispatcher() +{ + { + std::lock_guard lock(mutex); + is_cancelled = true; + auto exc = std::make_exception_ptr(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Dispatcher {} has been already destroyed", logger_name)); + try + { + cleanupLocked([&](const auto & binlog) + { + /// Notify the binlogs that the dispatcher is already destroyed + /// and it needs to recreate new binlogs if needed + binlog->setException(exc); + }); + } + catch (const std::exception & exc) + { + LOG_ERROR(logger, "Unexpected exception: {}", exc.what()); + } + } + cv.notify_all(); + if (dispatching_thread) + dispatching_thread->join(); +} + +static void flushTimers(Stopwatch & watch, UInt64 & total_time, UInt64 & size, float & size_per_sec, UInt64 & bytes, float & bytes_per_sec, float threshold_flush, float threshold_reset) +{ + total_time += watch.elapsedMicroseconds(); + const float elapsed_seconds = total_time * 1e-6f; + if (elapsed_seconds >= threshold_flush) + { + size_per_sec = size / elapsed_seconds; + bytes_per_sec = bytes / elapsed_seconds; + } + if (elapsed_seconds >= threshold_reset) + { + size = 0; + bytes = 0; + total_time = 0; + } +} + +void BinlogEventsDispatcher::flushBufferLocked() +{ + Stopwatch watch; + if (buffer.bytes) + cleanupLocked([&](const auto & b) { b->push(buffer); }); + events_flush += buffer.events.size(); + bytes_flush += buffer.bytes; + flushTimers(watch, events_flush_total_time, events_flush, events_flush_per_sec, bytes_flush, bytes_flush_per_sec, 0.1f, 1.0); + buffer = {}; +} + +static bool isDispatcherEventIgnored(const BinlogEventPtr & event) +{ + switch (event->header.type) + { + /// Sending to all databases: + case GTID_EVENT: /// Catch up requested executed gtid set, used only in BinlogFromDispatcher + case ROTATE_EVENT: /// Change binlog_checksum + case XID_EVENT: /// Commit transaction + /// Sending to all attached binlogs without filtering on dispatcher thread + /// to keep the connection as up-to-date as possible, + /// but these events should be filtered on databases' threads + /// and sent only to requested databases: + case QUERY_EVENT: /// Apply DDL + case WRITE_ROWS_EVENT_V1: /// Apply DML + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + return false; + default: + break; + } + return true; +} + +void BinlogEventsDispatcher::dispatchEvents() +{ + LOG_TRACE(logger, "{}: started", __FUNCTION__); + BinlogEventPtr event; + BinlogPtr binlog_; + Stopwatch watch; + UInt64 events_read = 0; + UInt64 bytes_read = 0; + UInt64 events_read_total_time = 0; + Stopwatch watch_events_read; + + while (!is_cancelled) + { + try + { + { + std::unique_lock lock(mutex); + cv.wait(lock, [&] { return is_cancelled || (binlog_read_from && !binlogs.empty()); }); + if (is_cancelled) + break; + + for (auto it = sync_to.begin(); it != sync_to.end() && !binlogs.empty();) + { + if (auto d = it->lock()) + { + /// If we can catch up the position of a dispatcher we synced to, + /// need to move all binlogs out + if (trySyncLocked(d)) + { + /// Don't keep connection longer than needed + stopLocked(); + break; + } + ++it; + } + else + { + it = sync_to.erase(it); + } + } + + if (binlog_read_from) + binlog_read_from->setChecksum(binlog_checksum); + binlog_ = binlog_read_from; + if (watch.elapsedMilliseconds() >= max_flush_ms || buffer.bytes >= max_bytes_in_buffer) + { + flushBufferLocked(); + watch.restart(); + } + } + + watch_events_read.restart(); + if (!is_cancelled && binlog_ && binlog_->tryReadEvent(event, max_flush_ms) && event) + { + ++events_read; + bytes_read += event->header.event_size; + { + std::lock_guard lock(mutex); + flushTimers(watch_events_read, events_read_total_time, events_read, events_read_per_sec, bytes_read, bytes_read_per_sec, 1.0, 5.0); + BinlogParser::updatePosition(event, position); + /// Ignore meaningless events + if (isDispatcherEventIgnored(event)) + continue; + buffer.events.push_back(event); + buffer.bytes += event->header.event_size; + buffer.position = position; + /// Deliver ROTATE event ASAP if there binlog_checksum should be changed + if (event->header.type == ROTATE_EVENT) + flushBufferLocked(); + } + } + } + catch (const std::exception & exc) + { + std::lock_guard lock(mutex); + LOG_ERROR(logger, "Exception: {}", exc.what()); + stopLocked(); + /// All attached binlogs should be recreated + cleanupLocked([&](const auto & b) { b->setException(std::current_exception()); }); + binlogs.clear(); + buffer = {}; + position = {}; + } + } + LOG_TRACE(logger, "{}: finished", __FUNCTION__); +} + +bool BinlogEventsDispatcher::cleanupLocked(const std::function & binlog)> & fn) +{ + for (auto it = binlogs.begin(); it != binlogs.end();) + { + if (auto binlog = it->lock()) + { + if (fn) + fn(binlog); + ++it; + } + else + { + it = binlogs.erase(it); + } + } + + return binlogs.empty(); +} + +bool BinlogEventsDispatcher::cleanupBinlogsAndStop() +{ + std::lock_guard lock(mutex); + const bool is_empty = cleanupLocked(); + if (is_empty && binlog_read_from) + stopLocked(); + return is_empty; +} + +void BinlogEventsDispatcher::stopLocked() +{ + if (!binlog_read_from) + { + LOG_DEBUG(logger, "Could not stop. Already stopped"); + return; + } + + cleanupLocked(); + binlog_read_from = nullptr; + LOG_DEBUG(logger, "Stopped: {}:{}.{}: ({})", position.binlog_name, position.gtid_sets.toString(), position.binlog_pos, getBinlogNames(binlogs)); +} + +BinlogPtr BinlogEventsDispatcher::createBinlogLocked(const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms, + const Position & pos_initial, + const Position & pos_wait) +{ + static int client_cnt = 0; + const String client_id = !name_.empty() ? name_ : "binlog_" + std::to_string(++client_cnt); + auto binlog = std::make_shared(client_id, mysql_database_names, max_bytes, max_waiting_ms); + binlogs.push_back(binlog); + binlog->setPosition(pos_initial, pos_wait); + LOG_DEBUG(logger, "Created binlog: {} -> {}", name_, binlog->getPosition().gtid_sets.toString()); + return binlog; +} + +BinlogPtr BinlogEventsDispatcher::start(const BinlogPtr & binlog_read_from_, + const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + BinlogPtr ret; + { + std::lock_guard lock(mutex); + if (is_started) + return {}; + binlog_read_from = binlog_read_from_; + /// It is used for catching up + /// binlog_read_from should return position with requested executed GTID set: 1-N + position = binlog_read_from->getPosition(); + ret = createBinlogLocked(name_, mysql_database_names, max_bytes, max_waiting_ms, position); + is_started = true; + } + cv.notify_all(); + return ret; +} + +BinlogPtr BinlogEventsDispatcher::attach(const String & executed_gtid_set, + const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + BinlogPtr ret; + { + std::lock_guard lock(mutex); + /// Check if binlog_read_from can be reused: + /// Attach to only active dispatchers + /// and if executed_gtid_set is higher value than current + if (!binlog_read_from || !is_started || cleanupLocked() || executed_gtid_set.empty()) + return {}; + Position pos_wait; + pos_wait.gtid_sets.parse(executed_gtid_set); + if (!BinlogParser::isNew(position, pos_wait)) + return {}; + ret = createBinlogLocked(name_, mysql_database_names, max_bytes, max_waiting_ms, position, pos_wait); + } + cv.notify_all(); + return ret; +} + +void BinlogEventsDispatcher::syncToLocked(const BinlogEventsDispatcherPtr & to) +{ + if (to && this != to.get()) + { + std::vector names; + for (const auto & it : sync_to) + { + if (auto dispatcher = it.lock()) + names.push_back(dispatcher->logger_name); + } + LOG_DEBUG(logger, "Syncing -> ({}) + ({})", boost::algorithm::join(names, ", "), to->logger_name); + sync_to.emplace_back(to); + } +} + +void BinlogEventsDispatcher::syncTo(const BinlogEventsDispatcherPtr & to) +{ + std::lock_guard lock(mutex); + syncToLocked(to); +} + +Position BinlogEventsDispatcher::getPosition() const +{ + std::lock_guard lock(mutex); + return position; +} + +bool BinlogEventsDispatcher::trySyncLocked(BinlogEventsDispatcherPtr & to) +{ + { + std::lock_guard lock(to->mutex); + /// Don't catch up if positions do not have GTIDs yet + const auto & cur_sets = position.gtid_sets.sets; + const auto & sets = to->position.gtid_sets.sets; + /// Sync to only started dispatchers + if (!to->binlog_read_from || (cur_sets.empty() && sets.empty()) || to->position != position) + return false; + + flushBufferLocked(); + to->flushBufferLocked(); + LOG_DEBUG(logger, "Synced up: {} -> {}: {}:{}.{}: ({}) + ({})", logger_name, to->logger_name, + position.binlog_name, position.gtid_sets.toString(), position.binlog_pos, getBinlogNames(to->binlogs), getBinlogNames(binlogs)); + std::move(binlogs.begin(), binlogs.end(), std::back_inserter(to->binlogs)); + } + + /// Notify that new binlogs arrived + to->cv.notify_all(); + return true; +} + +void BinlogEventsDispatcher::setBinlogChecksum(const String & checksum) +{ + std::lock_guard lock(mutex); + LOG_DEBUG(logger, "Setting binlog_checksum: {}", checksum); + binlog_checksum = IBinlog::checksumFromString(checksum); +} + +void BinlogFromDispatcher::push(const BinlogEventsDispatcher::Buffer & buffer_) +{ + std::unique_lock lock(mutex); + cv.wait_for(lock, std::chrono::milliseconds(max_waiting_ms), + [&] + { + bool ret = is_cancelled || exception || max_bytes == 0 || buffer.bytes < max_bytes; + if (!ret) + LOG_TRACE(logger, "Waiting: bytes: {} >= {}", buffer.bytes, max_bytes); + return ret; + }); + + if (is_cancelled || exception) + return; + + if (max_bytes != 0 && buffer.bytes >= max_bytes) + { + lock.unlock(); + setException(std::make_exception_ptr( + Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Timeout exceeded: Waiting: bytes: {} >= {}", buffer.bytes, max_bytes))); + return; + } + + auto it = buffer_.events.begin(); + size_t bytes = buffer_.bytes; + if (!gtid_sets_wait.sets.empty()) + { + if (!buffer_.position.gtid_sets.contains(gtid_sets_wait)) + { + LOG_TRACE(logger, "(wait_until: {} / {}) Skipped bytes: {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), buffer_.bytes); + return; + } + + std::vector seqs; + for (auto & s : gtid_sets_wait.sets) + { + GTID g; + g.uuid = s.uuid; + for (auto & in : s.intervals) + { + g.seq_no = in.end; + seqs.push_back(g); + } + } + for (; it != buffer_.events.end(); ++it) + { + const auto & event = *it; + auto find_if_func = [&](auto & a) + { + return std::static_pointer_cast(event)->gtid == a; + }; + if (event->header.type != GTID_EVENT || std::find_if(seqs.begin(), seqs.end(), find_if_func) == seqs.end()) + { + LOG_TRACE(logger, "(wait_until: {} / {}) Skipped {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), magic_enum::enum_name(event->header.type)); + bytes -= event->header.event_size; + continue; + } + LOG_DEBUG(logger, "(wait_until: {} / {}) Starting {}: gtid seq_no: {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), magic_enum::enum_name(event->header.type), + std::static_pointer_cast(event)->gtid.seq_no); + break; + } + gtid_sets_wait = {}; + } + + if (it != buffer_.events.end()) + { + std::copy(it, buffer_.events.end(), std::back_inserter(buffer.events)); + buffer.bytes += bytes; + buffer.position = buffer_.position; + } + lock.unlock(); + /// Notify that added some event + cv.notify_all(); +} + +static void rethrowIfNeeded(const std::exception_ptr & exception, size_t events_size) +{ + try + { + std::rethrow_exception(exception); + } + catch (const Exception & e) + { + /// If timeout exceeded, it is safe to read all events before rethrowning + if (e.code() == ErrorCodes::TIMEOUT_EXCEEDED && events_size > 0) + return; + throw; + } +} + +static bool isBinlogEventIgnored(const NameSet & mysql_database_names, const BinlogEventPtr & event) +{ + bool ret = false; + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + ret = !mysql_database_names.empty() && !mysql_database_names.contains(std::static_pointer_cast(event)->schema); + break; + case QUERY_EVENT: + if (event->type() != MYSQL_UNHANDLED_EVENT) + { + auto query_event = std::static_pointer_cast(event); + ret = !mysql_database_names.empty() && + !query_event->query_database_name.empty() && + !mysql_database_names.contains(query_event->query_database_name); + } + break; + default: + break; + } + return ret; +} + +bool BinlogFromDispatcher::tryReadEvent(BinlogEventPtr & to, UInt64 ms) +{ + auto wake_up_func = [&] + { + if (exception) + rethrowIfNeeded(exception, buffer.events.size()); + return is_cancelled || !buffer.events.empty(); + }; + to = nullptr; + std::unique_lock lock(mutex); + if (!cv.wait_for(lock, std::chrono::milliseconds(ms), wake_up_func) || is_cancelled || buffer.events.empty()) + return false; + to = buffer.events.front(); + buffer.events.pop_front(); + BinlogParser::updatePosition(to, position); + buffer.bytes -= to->header.event_size; + if (isBinlogEventIgnored(mysql_database_names, to)) + to = std::make_shared(EventHeader(to->header)); + lock.unlock(); + /// Notify that removed some event + cv.notify_all(); + return true; +} + +Position BinlogFromDispatcher::getPosition() const +{ + std::lock_guard lock(mutex); + return position; +} + +void BinlogFromDispatcher::setPosition(const Position & initial, const Position & wait) +{ + std::lock_guard lock(mutex); + if (wait.gtid_sets.sets.empty()) + { + position = initial; + } + else + { + position = wait; + gtid_sets_wait = wait.gtid_sets; + } +} + +void BinlogFromDispatcher::setException(const std::exception_ptr & exception_) +{ + { + std::lock_guard lock(mutex); + exception = exception_; + } + cv.notify_all(); +} + +BinlogEventsDispatcher::BinlogMetadata BinlogFromDispatcher::getBinlogMetadata() const +{ + std::lock_guard lock(mutex); + BinlogEventsDispatcher::BinlogMetadata ret; + ret.name = name; + ret.position_write = buffer.position; + ret.position_read = position; + ret.size = buffer.events.size(); + ret.bytes = buffer.bytes; + ret.max_bytes = max_bytes; + ret.max_waiting_ms = max_waiting_ms; + return ret; +} + +BinlogEventsDispatcher::DispatcherMetadata BinlogEventsDispatcher::getDispatcherMetadata() const +{ + std::lock_guard lock(mutex); + DispatcherMetadata ret; + ret.name = logger_name; + ret.position = position; + ret.events_read_per_sec = events_read_per_sec; + ret.bytes_read_per_sec = bytes_read_per_sec; + ret.events_flush_per_sec = events_flush_per_sec; + ret.bytes_flush_per_sec = bytes_flush_per_sec; + + for (const auto & it : binlogs) + { + if (auto binlog = it.lock()) + ret.binlogs.push_back(binlog->getBinlogMetadata()); + } + return ret; +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h new file mode 100644 index 00000000000..43379697015 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h @@ -0,0 +1,136 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB::MySQLReplication +{ + +class BinlogEventsDispatcher; +using BinlogEventsDispatcherPtr = std::shared_ptr; +class BinlogFromDispatcher; + +/** Reads the binlog events from one source and dispatches them over consumers. + * If it can catch up the position of the another dispatcher, it will move all consumers to this dispatcher. + */ +class BinlogEventsDispatcher final : boost::noncopyable +{ +public: + BinlogEventsDispatcher(const String & logger_name_ = "BinlogDispatcher", size_t max_bytes_in_buffer_ = 1_MiB, UInt64 max_flush_ms_ = 1000); + ~BinlogEventsDispatcher(); + + /// Moves all IBinlog objects to \a to if it has the same position + /// Supports syncing to multiple dispatchers + void syncTo(const BinlogEventsDispatcherPtr & to); + + /** Creates a binlog and starts the dispatching + * binlog_read_from Source binlog to read events from + * name Identifies the binlog, could be not unique + * mysql_database_names Returns events only from these databases + * max_bytes Defines a limit in bytes for this binlog + * Note: Dispatching will be stopped for all binlogs if bytes in queue increases this limit + * max_waiting_ms Max wait time when max_bytes exceeded + */ + BinlogPtr start(const BinlogPtr & binlog_read_from_, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /** Creates a binlog if the dispatcher is started + * executed_gtid_set Can be higher value than current, + * otherwise not possible to attach + * name Identifies the binlog, could be not unique + * mysql_database_names Returns events only from these databases + * max_bytes Defines a limit in bytes for this binlog + * max_waiting_ms Max wait time when max_bytes exceeded + */ + BinlogPtr attach(const String & executed_gtid_set, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /// Cleans the destroyed binlogs up and returns true if empty + bool cleanupBinlogsAndStop(); + + /// Changes binlog_checksum for binlog_read_from + void setBinlogChecksum(const String & checksum); + + Position getPosition() const; + + struct BinlogMetadata + { + String name; + /// Position that was written to + Position position_write; + /// Position that was read from + Position position_read; + size_t size = 0; + size_t bytes = 0; + size_t max_bytes = 0; + UInt64 max_waiting_ms = 0; + }; + struct DispatcherMetadata + { + String name; + Position position; + float events_read_per_sec = 0; + float bytes_read_per_sec = 0; + float events_flush_per_sec = 0; + float bytes_flush_per_sec = 0; + std::vector binlogs; + }; + DispatcherMetadata getDispatcherMetadata() const; + + struct Buffer + { + std::deque events; + size_t bytes = 0; + Position position; + }; + +private: + bool cleanupLocked(const std::function & binlog)> & fn = {}); + bool startLocked(const String & executed_gtid_set); + void stopLocked(); + BinlogPtr createBinlogLocked(const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0, + const Position & pos_initial = {}, + const Position & pos_wait = {}); + void syncToLocked(const BinlogEventsDispatcherPtr & to); + bool trySyncLocked(BinlogEventsDispatcherPtr & to); + void flushBufferLocked(); + void dispatchEvents(); + + const String logger_name; + const size_t max_bytes_in_buffer = 0; + const UInt64 max_flush_ms = 0; + Poco::Logger * logger = nullptr; + + BinlogPtr binlog_read_from; + + Position position; + std::vector> sync_to; + std::vector> binlogs; + std::atomic_bool is_cancelled{false}; + mutable std::mutex mutex; + std::condition_variable cv; + std::unique_ptr dispatching_thread; + IBinlog::Checksum binlog_checksum = IBinlog::CRC32; + bool is_started = false; + Buffer buffer; + float events_read_per_sec = 0; + float bytes_read_per_sec = 0; + UInt64 events_flush = 0; + UInt64 events_flush_total_time = 0; + float events_flush_per_sec = 0; + UInt64 bytes_flush = 0; + float bytes_flush_per_sec = 0; +}; + +} diff --git a/src/Databases/MySQL/tests/data/binlog.000016 b/src/Databases/MySQL/tests/data/binlog.000016 new file mode 100644 index 0000000000000000000000000000000000000000..e27a2bac9ff7ea59f8ed3821251c425583186360 GIT binary patch literal 1237 zcmeyDl$lr2T${wt$iTo*1;lkg%)r3HV4-KAXJn~sP?}VlS5gXO8|WFB=oy1$aTCG} zybK%=16Y7m5(A4Ah|9sj!33nZxVW^m7=%n1xHK3TMcUeSfNWL<+3XI)M}fG30mK7= zl{fmE5Bdqd*=?rc{qKHx8cZBUD}l@hg4(9j3pVJ~B`Ja={DD{mq5@q%Og+dQU;s0y zCoBKx!+WfMd>H3iFfv@{59kLO#{`m&1>!;=gM&dEq6g?Z1~mwiQGsEBD~QR)TwIdE z@}B_&Ibnth{bwl7ODrhP$S;X6Ni8m6aB_9`^!vZ}$~2JS!XRbVK)eR5J|1KY(Br(Q zS{M@kXOB`gdf$`}~>)^*H4gyp$V=YEiJ z@UY|r>Z4XzG78`M4Kf@amM&mN!2$$dx^zqx_zcnt56dnTt%$G$3B!U8%wRnK(XLYgHT~L| literal 0 HcmV?d00001 diff --git a/src/Databases/MySQL/tests/data/binlog.001390 b/src/Databases/MySQL/tests/data/binlog.001390 new file mode 100644 index 0000000000000000000000000000000000000000..1fbaae08a986438bd55a509dddaa62b59e9e8ec8 GIT binary patch literal 3091 zcmb`~4@{J090%~Tp9z)qB7ld+Cph-YP;_dNF=7dD44d)|M~ z@A*D2pFQthm~_8XH*WB{?Fs;F!?qn8fC&nnsxz-JR}HuWb^ZXJt4=lF>Ehn$cPTIz zELwmG4{k7J(51y!wXRqJ$@ySi4oX*Q!9O(fGMaes0eM&pK)R%tm?k2B$30-c&ma5&KI$qS%bMPjD)Prq`;c7x?f%0LY z0_)CPvs0Xbx{I-G#uE!X^0YPxM`vnh%3?TDM5nWiK7X|-4D^#6hH5uFT(qIM^gCbp zH3~|hAQjuYJmFp*u7CmSlqxV8af3uffs1*Y4pIFgD$B>#7h4ZF1>MoNqBlPIg2voM zHyzkUc+9}|6RYt8-s-aL&Dp7=xnKL`r_$UVq2%A01`4A+86`HNDPWo(3a(C29G8`0 z**jPlLz7K|@e%)vW;Q<>_UeoR6X%C0Q5qfRnOmnb>N)l8KB{+6)_QDz$JV1!JQ9mh z9Sx#(;C=gPN|9=5M!XBk4oz+U!IMie4(rrvxI1n^rU^BE1`N>304lBrdW=%dQFi!I zij!(5v2I3cOK%P@rbMY0{aG_ zC`GFEa*C}{dqV>+%yW6K>(tC0)Y zvO20LQL63c2Rty{8@hb>%6zJqYUg6>rP?jm+F(}qMM{xsW1P|zQET5ec41*qr?!?q z)NyJVLQQCK3aZ`Lx(+O;R?1(4U*5c2NpY-Jh20-vTvROyARBMuru^U6pvvv7_t#Wc zdIGhzmA;0$E!qcE4GwIi2^g^igMQ`bZHyo$#Sb(3-L?2dwcC>uLdKBxSv43#VK~vT zD~+Pqm%go;vvCI^%0Rw}WIKm;Ch*tz{N0^_2WgDdT*G6IM>H?^-tK?prcSekyW=z$ z2u-1=p#zK`G3_ADa@I_2`R!|plbWL#cPBL+zNW8e0;##mswcP|&8+5zZzzg2=?{(i zm?+lNLGoz=o92xw-#<=6CF(vNIv7Fy^4X>rd;ip-j&OGz>Oz4kFg1ICa!-0Yp;jQr6oU#)X# ytW*v1*rAB(qBB2t1fG1=ZKB1UgDqta?r@oF8~y`O>)q;9)s?OXRt`O9?)(=kJyBQy literal 0 HcmV?d00001 diff --git a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp new file mode 100644 index 00000000000..df8433f7cce --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp @@ -0,0 +1,1754 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + +using namespace DB; +using namespace DB::MySQLReplication; + +#define TRY_LOOP_IMPL(expr, timeout) \ + const unsigned long _test_step = (timeout) < 350 ? (timeout) / 7 + 1 : 50; \ + for (int _i = 0; _i < (timeout) && !(expr); _i += _test_step) \ + std::this_thread::sleep_for(std::chrono::milliseconds(_test_step)); \ + +#define TRY_ASSERT_EQ(expr, expected, timeout) \ +do { \ + TRY_LOOP_IMPL(((expr) == (expected)), timeout) \ + ASSERT_EQ((expr), expected); \ +} while (false) + +#define TRY_ASSERT_TRUE(expr, timeout) \ + TRY_ASSERT_EQ((expr), true, timeout) + +static std::string getTestDataRoot() +{ + static auto root = []() -> std::string + { + std::filesystem::path testdata_path("src/Databases/MySQL/tests/data"); + auto basedir = std::filesystem::current_path(); + while (basedir != basedir.parent_path()) + { + if (std::filesystem::exists(basedir / testdata_path)) + { + testdata_path = basedir / testdata_path; + break; + } + basedir = basedir.parent_path(); + } + auto path = basedir / testdata_path; + return std::filesystem::exists(path) ? path.string() : ""; + }(); + return root; +} + +static String getTestDataPath(const String & testdata_file) +{ + return (std::filesystem::path(getTestDataRoot()) / testdata_file).string(); +} + +class MySQLBinlog : public ::testing::Test +{ +protected: + void SetUp() override + { + if (getTestDataRoot().empty()) + GTEST_SKIP() << "Skipping all tests since no test data files found"; + } + + UInt64 timeout = 25000; +}; + +TEST_F(MySQLBinlog, positionEndLogPosOverflow) +{ + Position position; + EventHeader header; + header.event_size = 8161; + header.log_pos = 4294958114; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + ASSERT_TRUE(position.binlog_name.empty()); + ASSERT_TRUE(position.gtid_sets.toString().empty()); + ASSERT_EQ(position.timestamp, 0); + + header.log_pos = 4294966149; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + UInt64 prev = position.binlog_pos; + + header.log_pos = 7014; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + prev = position.binlog_pos; + + header.event_size = 8107; + header.log_pos = 15121; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + prev = position.binlog_pos; + + header.event_size = 8131; + header.log_pos = 23252; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position.binlog_pos = 4724501662; + prev = position.binlog_pos; + + header.event_size = 8125; + header.log_pos = 429542491; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position.binlog_pos = 5474055640; + prev = position.binlog_pos; + + header.event_size = 31; + header.log_pos = 1179088375; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position = {}; + header.log_pos = 4294965445; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + prev = position.binlog_pos; + + header.event_size = 7927; + header.log_pos = 6076; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); +} + +TEST_F(MySQLBinlog, positionEquals) +{ + Position p1; + Position p2; + ASSERT_EQ(p1, p2); + p1.binlog_pos = 1; + ASSERT_NE(p1, p2); + p2.binlog_pos = 1; + ASSERT_EQ(p1, p2); + p1.gtid_sets.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + ASSERT_NE(p1, p2); + p2.gtid_sets.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + ASSERT_EQ(p1, p2); + p1.binlog_name = "name"; + ASSERT_NE(p1, p2); + p2.binlog_name = "name"; + ASSERT_EQ(p1, p2); +} + +TEST_F(MySQLBinlog, positionMultimaster) +{ + Position p1; + Position p2; + p1.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + ASSERT_TRUE(p1.gtid_sets.contains(p2.gtid_sets)); + ASSERT_FALSE(p2.gtid_sets.contains(p1.gtid_sets)); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:1-10"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:171-172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:171-173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx:173"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); +} + +static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) +{ + BinlogEventPtr event; + int count = 0; + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, FORMAT_DESCRIPTION_EVENT); + ASSERT_EQ(event->header.timestamp, 1651442421); + ASSERT_EQ(event->header.event_size, 122); + ASSERT_EQ(event->header.log_pos, 126); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, PREVIOUS_GTIDS_EVENT); + ASSERT_EQ(event->header.timestamp, 1651442421); + ASSERT_EQ(event->header.event_size, 71); + ASSERT_EQ(event->header.log_pos, 197); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 276); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 73); + ASSERT_EQ(event->header.log_pos, 349); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 397); + + auto table_event = std::static_pointer_cast(event); + ASSERT_TRUE(table_event); + ASSERT_EQ(table_event->table_id, 7566); + ASSERT_EQ(table_event->flags, 1); + ASSERT_EQ(table_event->schema_len, 2u); + ASSERT_EQ(table_event->schema, "db"); + ASSERT_EQ(table_event->table_len, 1u); + ASSERT_EQ(table_event->table, "a"); + ASSERT_EQ(table_event->column_count, 4); + std::vector column_type = {3u, 3u, 3u, 3u}; + ASSERT_EQ(table_event->column_type, column_type); + std::vector column_meta = {0, 0, 0, 0}; + ASSERT_EQ(table_event->column_meta, column_meta); + std::vector column_charset = {}; + ASSERT_EQ(table_event->column_charset, column_charset); + ASSERT_EQ(table_event->default_charset, 255u); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, WRITE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 52); + ASSERT_EQ(event->header.log_pos, 449); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(event); + auto write_event = std::static_pointer_cast(event); + ASSERT_TRUE(write_event); + ASSERT_EQ(write_event->number_columns, 4); + ASSERT_EQ(write_event->schema, "db"); + ASSERT_EQ(write_event->table, "a"); + ASSERT_EQ(write_event->rows.size(), 1); + ASSERT_EQ(write_event->rows[0].getType(), Field::Types::Tuple); + auto row_data = write_event->rows[0].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 1u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 480); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 559); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 641); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 689); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.log_pos, 759); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(event); + auto update_event = std::static_pointer_cast(event); + ASSERT_TRUE(update_event); + ASSERT_EQ(update_event->number_columns, 4); + ASSERT_EQ(update_event->schema, "db"); + ASSERT_EQ(update_event->table, "a"); + ASSERT_EQ(update_event->rows.size(), 2); + ASSERT_EQ(update_event->rows[0].getType(), Field::Types::Tuple); + row_data = update_event->rows[0].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 1u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + row_data = update_event->rows[1].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 2u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 790); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 869); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 951); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 999); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.log_pos, 1069); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 1100); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 1179); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 1261); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 1309); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.log_pos, 1379); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87830"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 1410); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87831"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 1489); + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87832"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87831"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 1571); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 1619); + } + + int total_count = filtered ? 37 : 48; + for (; count < total_count; ++count) + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.timestamp, 1651528821); + ASSERT_EQ(event->header.event_size, 44); + ASSERT_EQ(event->header.log_pos, 3091); + ASSERT_EQ(count, total_count); + ASSERT_FALSE(binlog.tryReadEvent(event, 10)); + + auto position = binlog.getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); +} + +TEST_F(MySQLBinlog, binlogFromFile1) +{ + BinlogFromFile binlog; + binlog.open(getTestDataPath("binlog.001390")); + testFile1(binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromFactory1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto binlog = f->createBinlog(""); + + testFile1(*binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromFactory1ExecutedGtidSet) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + BinlogEventPtr event; + + auto binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 559); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87829"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 8; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + auto position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); + + binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87829"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 869); + + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87830"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 13; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); + + binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87834"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 2443); + + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87835"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 38; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); +} + +TEST_F(MySQLBinlog, binlogFromDispatcher1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + BinlogEventsDispatcher d; + auto b = d.start(f->createBinlog("")); + testFile1(*b, timeout, true); + ASSERT_EQ(d.getPosition().gtid_sets.toString(), b->getPosition().gtid_sets.toString()); +} + +static void testFile2(IBinlog & binlog, UInt64 timeout, bool filtered = false) +{ + BinlogEventPtr event; + int count = 0; + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, FORMAT_DESCRIPTION_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, PREVIOUS_GTIDS_EVENT); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 608); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:1059"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.log_pos, 701); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.log_pos, 760); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.log_pos, 830); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.log_pos, 861); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 940); + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:1060"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + int total_count = filtered ? 13 : 18; + for (; count < total_count; ++count) + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.log_pos, 1237); + ASSERT_EQ(count, total_count); + ASSERT_FALSE(binlog.tryReadEvent(event, 10)); + + auto position = binlog.getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.000017"); + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, binlogFromFile2) +{ + BinlogFromFile binlog; + binlog.open(getTestDataPath("binlog.000016")); + testFile2(binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromDispatcher2) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + BinlogEventsDispatcher d; + auto b = d.start(f->createBinlog("")); + testFile2(*b, timeout, true); + ASSERT_EQ(d.getPosition().gtid_sets.toString(), b->getPosition().gtid_sets.toString()); +} + +TEST_F(MySQLBinlog, binlogsFromOneFile) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); +} + +TEST_F(MySQLBinlog, empty) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); +} + +TEST_F(MySQLBinlog, binlogsAfterStart) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d1->start(f->createBinlog("")); + ASSERT_FALSE(b2); + + testFile2(*b1, timeout, true); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, metadata) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); + ASSERT_EQ(d1->getDispatcherMetadata().name, "d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + + TRY_ASSERT_TRUE(!d1->getDispatcherMetadata().position.gtid_sets.sets.empty(), timeout); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + + testFile2(*b1, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); +} + +TEST_F(MySQLBinlog, catchingUp) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); + ASSERT_TRUE(d2->getDispatcherMetadata().binlogs.empty()); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + ASSERT_TRUE(b1); + ASSERT_TRUE(b2); + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_FALSE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(d2->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_FALSE(d2->start(f->createBinlog(""))); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].bytes, 0); +} + +TEST_F(MySQLBinlog, catchingUpFastMaster) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + std::this_thread::sleep_for(std::chrono::milliseconds(50)); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); +} + +TEST_F(MySQLBinlog, catchingUpFastSlave) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b2 = d2->start(f->createBinlog("")); + std::this_thread::sleep_for(std::chrono::milliseconds(50)); + auto b1 = d1->start(f->createBinlog("")); + + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_FALSE(d2->getDispatcherMetadata().position.gtid_sets.sets.empty()); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); +} + +TEST_F(MySQLBinlog, catchingUpWithoutWaiting) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + TRY_ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0, timeout); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.binlog_pos, 4); +} + +TEST_F(MySQLBinlog, catchingUpManyToOne) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d0 = std::make_shared("d0"); + std::vector ds; + int n = 10; + for (int i = 0; i < n; ++i) + { + auto d = std::make_shared("r" + std::to_string(i)); + d->syncTo(d0); + ds.push_back(d); + } + + for (int i = 0; i < n; ++i) + ASSERT_TRUE(ds[i]->getDispatcherMetadata().binlogs.empty()); + + auto b0 = d0->start(f->createBinlog(""), "b"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 0); + std::vector bs; + bs.resize(n); + for (int i = 0; i < n; ++i) + bs[i] = ds[i]->start(f->createBinlog(""), "b" + std::to_string(i)); + + TRY_ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), n + 1, timeout); + ASSERT_FALSE(d0->getDispatcherMetadata().position.gtid_sets.sets.empty()); + for (int i = 0; i < n; ++i) + { + ASSERT_EQ(ds[i]->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(ds[i]->getDispatcherMetadata().position.gtid_sets.sets.empty()); + } + + testFile2(*b0, timeout, true); + for (int i = 0; i < n; ++i) + testFile2(*bs[i], timeout, true); + + ASSERT_EQ(b0->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b0->getPosition().binlog_pos, 4); + + for (int i = 0; i < n; ++i) + { + ASSERT_EQ(bs[i]->getPosition().gtid_sets.toString(), b0->getPosition().gtid_sets.toString()); + ASSERT_EQ(bs[i]->getPosition().binlog_pos, b0->getPosition().binlog_pos); + } + + for (int i = 0; i < n; ++i) + ASSERT_EQ(ds[i]->getDispatcherMetadata().binlogs.size(), 0); + + ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), n + 1); + for (int i = 0; i < n + 1; ++i) + { + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_read.binlog_pos, 4); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_write.binlog_pos, 4); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].size, 0); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].bytes, 0); + } +} + +TEST_F(MySQLBinlog, catchingUpStopApplier) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + d1 = nullptr; + + auto b2 = d2->start(f->createBinlog("")); + ASSERT_TRUE(b2); + testFile2(*b2, timeout, true); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, catchingUpOneToAllPrevious) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + std::vector ds; + int n = 10; + for (int i = 0; i < n; ++i) + { + auto d = std::make_shared("d" + std::to_string(i)); + for (int j = 0; j < i; ++j) + d->syncTo(ds[j]); + ds.push_back(d); + } + + for (int i = 0; i < n; ++i) + ASSERT_TRUE(ds[i]->getDispatcherMetadata().binlogs.empty()); + + std::vector bs; + bs.resize(n); + for (int i = 0; i < n; ++i) + bs[i] = ds[i]->start(f->createBinlog(""), "b" + std::to_string(i)); + + auto check_dispatchers = [&] + { + int not_empty_count = 0; + int ii = 0; + for (int i = 0; i < n; ++i) + { + if (!ds[i]->getDispatcherMetadata().binlogs.empty()) + { + ++not_empty_count; + ii = i; + } + } + return not_empty_count == 1 && ds[ii]->getDispatcherMetadata().binlogs.size() == n; + }; + + for (int i = 0; i < n; ++i) + testFile2(*bs[i], timeout, true); + + TRY_ASSERT_TRUE(check_dispatchers(), timeout); + + for (int i = 1; i < n; ++i) + { + ASSERT_EQ(bs[i]->getPosition().gtid_sets.toString(), bs[0]->getPosition().gtid_sets.toString()); + ASSERT_EQ(bs[i]->getPosition().binlog_pos, bs[0]->getPosition().binlog_pos); + } + + int i = 0; + for (int j = 0; j < n; ++j) + { + auto bs_ = ds[j]->getDispatcherMetadata().binlogs; + for (; i < bs_.size(); ++i) + { + ASSERT_EQ(bs_[i].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(bs_[i].position_write.gtid_sets.toString(), bs_[i].position_write.gtid_sets.toString()); + ASSERT_EQ(bs_[i].position_read.binlog_pos, 4); + ASSERT_EQ(bs_[i].position_write.binlog_pos, 4); + ASSERT_EQ(bs_[i].size, 0); + ASSERT_EQ(bs_[i].bytes, 0); + } + } + ASSERT_EQ(i, n); +} + +TEST_F(MySQLBinlog, catchingUpMaxBytes) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog(""), "big"); + auto b2 = d2->start(f->createBinlog(""), "small", {}, 1, 10000); + + testFile2(*b2, timeout, true); + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_EQ(d1->getDispatcherMetadata().position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + testFile2(*b1, timeout, true); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); +} + +TEST_F(MySQLBinlog, filterEvents) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto b1 = d1->start(f->createBinlog(""), "b1", {"db"}); + auto b2 = d2->start(f->createBinlog(""), "b2", {"unknown_database"}); + + BinlogEventPtr event; + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "db"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(b1->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case ROTATE_EVENT: + case XID_EVENT: + case QUERY_EVENT: + case GTID_EVENT: + break; + default: + if (event->type() != MYSQL_UNHANDLED_EVENT) + FAIL() << "Unexpected event: " << magic_enum::enum_name(event->header.type); + break; + } + } + + ASSERT_FALSE(b2->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), b1->getPosition().gtid_sets.toString()); + ASSERT_EQ(b2->getPosition().binlog_pos, b1->getPosition().binlog_pos); + ASSERT_FALSE(b2->tryReadEvent(event, 0)); +} + +TEST_F(MySQLBinlog, filterEventsMultipleDatabases) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto d3 = std::make_shared("d3"); + auto d4 = std::make_shared("d4"); + auto d5 = std::make_shared("d5"); + auto all_dbs = d1->start(f->createBinlog(""), "all_dbs"); + auto db = d2->start(f->createBinlog(""), "db", {"db"}); + auto aborted = d3->start(f->createBinlog(""), "aborted_full_sync", {"aborted_full_sync"}); + auto db_and_aborted = d4->start(f->createBinlog(""), "db_and_aborted", {"db", "aborted_full_sync"}); + auto unknown = d5->start(f->createBinlog(""), "unknown", {"unknown1", "unknown2"}); + + BinlogEventPtr event; + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(all_dbs->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + break; + default: + break; + } + } + + ASSERT_FALSE(all_dbs->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(db->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "db"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(db->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(aborted->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "aborted_full_sync"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(aborted->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(db_and_aborted->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + { + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + auto schema = std::static_pointer_cast(event)->schema; + ASSERT_TRUE(schema == "db" || schema == "aborted_full_sync"); + } break; + default: + break; + } + } + + ASSERT_FALSE(db_and_aborted->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(unknown->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case ROTATE_EVENT: + case XID_EVENT: + case QUERY_EVENT: + case GTID_EVENT: + break; + default: + ASSERT_EQ(event->type(), MYSQL_UNHANDLED_EVENT); + break; + } + } + + ASSERT_FALSE(unknown->tryReadEvent(event, 0)); + ASSERT_FALSE(event); +} + +TEST_F(MySQLBinlog, dispatcherStop) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + d1 = nullptr; + BinlogEventPtr event; + EXPECT_THROW(for (int i = 0; i < 18 + 1; ++i) b1->tryReadEvent(event, timeout), DB::Exception); +} + +TEST_F(MySQLBinlog, executedGTIDSet) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1058"), "b1"); + + BinlogEventPtr event; + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 608); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.log_pos, 701); + + for (int i = 0; i < 7; ++i) + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.log_pos, 1237); + ASSERT_EQ(d1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1060"); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1060"); + ASSERT_FALSE(b1->tryReadEvent(event, 0)); +} + +TEST_F(MySQLBinlog, client) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto e = std::make_shared(f); + + auto b1 = e->createBinlog("", "b1"); + auto b2 = e->createBinlog("", "b2"); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + auto b3 = e->createBinlog("", "b3"); + + testFile2(*b3, timeout, true); + + b1 = nullptr; + b2 = nullptr; + + auto b4 = e->createBinlog("", "b4"); + testFile2(*b4, timeout, true); + + b3 = nullptr; + b4 = e->createBinlog("", "b4 2"); + testFile2(*b4, timeout, true); + + b1 = e->createBinlog("", "b1 2"); + b2 = e->createBinlog("", "b2 2"); + testFile2(*b1, timeout, true); + + b3 = e->createBinlog("", "b3 2"); + testFile2(*b2, timeout, true); + + b4 = e->createBinlog("", "b4 3"); + testFile2(*b3, timeout, true); + testFile2(*b4, timeout, true); + + b1 = nullptr; + b2 = nullptr; + b3 = nullptr; + b4 = nullptr; + b1 = e->createBinlog("", "b1 3"); + b2 = e->createBinlog("", "b2 3"); + b3 = e->createBinlog("", "b3 3"); + b4 = e->createBinlog("", "b4 4"); + testFile2(*b4, timeout, true); + testFile2(*b3, timeout, true); + testFile2(*b2, timeout, true); + testFile2(*b1, timeout, true); + + f = std::make_shared(getTestDataPath("binlog.000016")); + e = std::make_shared(f); + + b4 = e->createBinlog("", "b4 5"); + b3 = e->createBinlog("", "b3 4"); + testFile2(*b4, timeout, true); + b2 = e->createBinlog("", "b2 4"); + b1 = e->createBinlog("", "b1 4"); + testFile2(*b3, timeout, true); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + b1 = e->createBinlog("", "b1 5"); + b2 = e->createBinlog("", "b2 5"); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + b1 = e->createBinlog("", "b1 6"); + testFile2(*b1, timeout, true); + b1 = e->createBinlog("", "b1 7"); + testFile2(*b1, timeout, true); + + b3 = nullptr; + b4 = nullptr; + b1 = e->createBinlog("", "b1 8"); + b4 = e->createBinlog("", "b4 6"); + b3 = e->createBinlog("", "b3 5"); + testFile2(*b4, timeout, true); + testFile2(*b3, timeout, true); + testFile2(*b1, timeout, true); + + b2 = nullptr; + b3 = nullptr; + b4 = nullptr; + b1 = nullptr; + b1 = e->createBinlog("", "b1 9"); + testFile2(*b1, timeout, true); +} + +TEST_F(MySQLBinlog, createBinlog) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog(""), "b1"); + ASSERT_TRUE(b1); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + testFile1(*b1, timeout, true); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + b1 = nullptr; + ASSERT_FALSE(d1->start(f->createBinlog(""))); +} + +TEST_F(MySQLBinlog, createBinlogAttach1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto b1_ = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b1_"); + ASSERT_TRUE(b1_); + auto b1 = d1->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831", "b1"); + if (b1) + { + BinlogEventPtr event; + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87832"); + for (int i = 0; i < 17; ++i) + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b1->tryReadEvent(event, 10)); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(b1->getPosition().binlog_name, "binlog.001391"); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b1_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].bytes, 0); + } +} + +TEST_F(MySQLBinlog, createBinlogAttach2) +{ + BinlogEventPtr event; + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto d3 = std::make_shared("d3"); + auto d4 = std::make_shared("d4"); + + auto b1 = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b1"); + ASSERT_TRUE(b1); + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + auto b2_ = d2->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b2_"); + ASSERT_TRUE(b2_); + auto b2 = d2->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831", "b2"); + + auto b3_ = d3->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b3_"); + ASSERT_TRUE(b3_); + auto b3 = d3->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87835", "b3"); + + auto b4_ = d4->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b4_"); + ASSERT_TRUE(b4_); + auto b4 = d4->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828", "b4"); + + /// There is a race with dispatcher thread + if (b2) + { + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + for (int i = 0; i < 18; ++i) + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b2->tryReadEvent(event, 10)); + ASSERT_EQ(b2->getPosition().binlog_pos, 4); + ASSERT_EQ(b2->getPosition().binlog_name, "binlog.001391"); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b2_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs[1].bytes, 0); + } + + if (b4) + { + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, WRITE_ROWS_EVENT_V2); + for (int i = 0; i < 10; ++i) + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b2->tryReadEvent(event, 10)); + ASSERT_EQ(b4->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b4_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs[1].bytes, 0); + } + + if (b3) + { + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + for (int i = 0; i < 3; ++i) + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b3->tryReadEvent(event, 10)); + ASSERT_EQ(b3->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b3_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs[1].bytes, 0); + } +} + +TEST_F(MySQLBinlog, factoryThreads) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto func1 = [&] + { + auto b1 = f->createBinlog(""); + auto b2 = f->createBinlog(""); + auto b3 = f->createBinlog(""); + testFile1(*b1, timeout); + testFile1(*b2, timeout); + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b1 = nullptr; + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + testFile1(*b3, timeout); + }; + + auto func2 = [&] + { + auto b1 = f->createBinlog(""); + auto b2 = f->createBinlog(""); + testFile1(*b2, timeout); + testFile1(*b1, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + b2 = f->createBinlog(""); + testFile1(*b1, timeout); + b2 = nullptr; + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b1 = nullptr; + }; + + int n = 4; + std::vector ts1, ts2; + for (int i = 0; i < n; ++i) + { + ts1.emplace_back(std::thread(func1)); + ts2.emplace_back(std::thread(func2)); + } + for (int i = 0; i < n; ++i) + { + ts1[i].join(); + ts2[i].join(); + } +} + +TEST_F(MySQLBinlog, clientThreads) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto e = std::make_shared(f); + auto func1 = [&] + { + auto b1 = e->createBinlog(""); + auto b2 = e->createBinlog(""); + testFile1(*b1, timeout, true); + testFile1(*b2, timeout, true); + b1 = nullptr; + b2 = nullptr; + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + b1 = nullptr; + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b2 = nullptr; + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + }; + + auto func2 = [&] + { + auto b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + auto b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b2 = e->createBinlog(""); + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + testFile1(*b2, timeout, true); + b1 = nullptr; + b2 = nullptr; + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + }; + + int n = 4; + std::vector ts1, ts2; + for (int i = 0; i < n; ++i) + { + ts1.emplace_back(std::thread(func1)); + ts2.emplace_back(std::thread(func2)); + } + for (int i = 0; i < n; ++i) + { + ts1[i].join(); + ts2[i].join(); + } + + // All dispatchers synced and finished + // No dispatchers and no binlogs are alive here + ASSERT_EQ(e->getMetadata().dispatchers.size(), 0); + + // Creates new dispatcher + auto b1 = e->createBinlog("", "b1 1"); + testFile1(*b1, timeout, true); + + auto md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(md[0].binlogs.size(), 1); + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(md[0].binlogs[0].size, 0); + ASSERT_EQ(md[0].binlogs[0].bytes, 0); + + // Creates new dispatcher + auto b1_2 = e->createBinlog("", "b1 2"); + + // Should sync to the first dispatcher + TRY_ASSERT_EQ(e->getMetadata().dispatchers.size(), 1, timeout); + // If there is no CPU available, + // it possible to catch in the middle of the transform between dispatchers. + // Checking again to make sure that catching up is finished. + TRY_ASSERT_EQ(e->getMetadata().dispatchers.size(), 1, timeout); + b1 = nullptr; + md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].binlogs.size(), 1); + // Did not read any events yet + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[0].position_read.binlog_pos, 0); + + auto b2 = e->createBinlog("", "b2"); + + BinlogEventPtr event; + // Read only one event + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + // Waits before all binlogs are moved to main dispatcher + TRY_ASSERT_EQ(e->getMetadata().dispatchers[0].binlogs.size(), 2, timeout); + + // One dispatcher is alive + md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].binlogs.size(), 2); + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[1].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[0].position_read.binlog_pos, md[0].binlogs[0].name == "b2" ? 276 : 0); // Read one event + ASSERT_EQ(md[0].binlogs[1].position_read.binlog_pos, md[0].binlogs[0].name == "b2" ? 0 : 276); +} diff --git a/src/Storages/System/StorageSystemMySQLBinlogs.cpp b/src/Storages/System/StorageSystemMySQLBinlogs.cpp new file mode 100644 index 00000000000..32648d22ee8 --- /dev/null +++ b/src/Storages/System/StorageSystemMySQLBinlogs.cpp @@ -0,0 +1,164 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemMySQLBinlogs::getNamesAndTypes() +{ + return { + {"binlog_client_name", std::make_shared()}, + {"name", std::make_shared()}, + {"mysql_binlog_name", std::make_shared()}, + {"mysql_binlog_pos", std::make_shared()}, + {"mysql_binlog_timestamp", std::make_shared()}, + {"mysql_binlog_executed_gtid_set", std::make_shared()}, + {"dispatcher_name", std::make_shared()}, + {"dispatcher_mysql_binlog_name", std::make_shared()}, + {"dispatcher_mysql_binlog_pos", std::make_shared()}, + {"dispatcher_mysql_binlog_timestamp", std::make_shared()}, + {"dispatcher_mysql_binlog_executed_gtid_set", std::make_shared()}, + {"size", std::make_shared()}, + {"bytes", std::make_shared()}, + {"max_bytes", std::make_shared()}, + {"max_waiting_ms", std::make_shared()}, + {"dispatcher_events_read_per_sec", std::make_shared()}, + {"dispatcher_bytes_read_per_sec", std::make_shared()}, + {"dispatcher_events_flush_per_sec", std::make_shared()}, + {"dispatcher_bytes_flush_per_sec", std::make_shared()}, + }; +} + +StorageSystemMySQLBinlogs::StorageSystemMySQLBinlogs(const StorageID & storage_id_) + : IStorage(storage_id_) +{ + StorageInMemoryMetadata storage_metadata; + ColumnsDescription columns(getNamesAndTypes()); + storage_metadata.setColumns(columns); + setInMemoryMetadata(storage_metadata); +} + +class MetadataSource : public ISource +{ +public: + using DispatcherMetadata = MySQLReplication::BinlogEventsDispatcher::DispatcherMetadata; + using BinlogMetadata = MySQLReplication::BinlogEventsDispatcher::BinlogMetadata; + + MetadataSource(Block block_header_, const std::vector & clients_) + : ISource(block_header_) + , block_to_fill(std::move(block_header_)) + , clients(clients_) + {} + + String getName() const override { return "MySQLBinlogClient"; } + +protected: + Chunk generate() override + { + if (clients.empty()) + return {}; + + Columns columns; + columns.reserve(block_to_fill.columns()); + + size_t total_size = 0; + auto create_column = [&](auto && column, const std::function & field) + { + size_t size = 0; + for (const auto & client : clients) + { + for (const auto & d : client.dispatchers) + { + for (const auto & b : d.binlogs) + { + column->insert(field(client.binlog_client_name, d, b)); + ++size; + } + } + } + if (!total_size) + total_size = size; + return std::forward(column); + }; + + for (const auto & elem : block_to_fill) + { + if (elem.name == "binlog_client_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto n, auto, auto) { return Field(n); })); + else if (elem.name == "name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.name); })); + else if (elem.name == "mysql_binlog_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.position_read.binlog_name); })); + else if (elem.name == "mysql_binlog_pos") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.position_read.binlog_pos); })); + else if (elem.name == "mysql_binlog_timestamp") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.position_read.timestamp); })); + else if (elem.name == "mysql_binlog_executed_gtid_set") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.position_read.gtid_sets.toString()); })); + else if (elem.name == "dispatcher_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.name); })); + else if (elem.name == "dispatcher_mysql_binlog_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.position.binlog_name); })); + else if (elem.name == "dispatcher_mysql_binlog_pos") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto d, auto) { return Field(d.position.binlog_pos); })); + else if (elem.name == "dispatcher_mysql_binlog_timestamp") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto d, auto) { return Field(d.position.timestamp); })); + else if (elem.name == "dispatcher_mysql_binlog_executed_gtid_set") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.position.gtid_sets.toString()); })); + else if (elem.name == "size") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.size); })); + else if (elem.name == "bytes") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.bytes); })); + else if (elem.name == "max_bytes") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.max_bytes); })); + else if (elem.name == "max_waiting_ms") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.max_waiting_ms); })); + else if (elem.name == "dispatcher_events_read_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.events_read_per_sec); })); + else if (elem.name == "dispatcher_bytes_read_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.bytes_read_per_sec); })); + else if (elem.name == "dispatcher_events_flush_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.events_flush_per_sec); })); + else if (elem.name == "dispatcher_bytes_flush_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.bytes_flush_per_sec); })); + } + + clients.clear(); + return {std::move(columns), total_size}; + } + +private: + Block block_to_fill; + std::vector clients; +}; + +Pipe StorageSystemMySQLBinlogs::read( + 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_); + const ColumnsDescription & our_columns = storage_snapshot->getDescriptionForColumns(column_names_); + Block block_header; + for (const auto & name : column_names_) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + return Pipe{std::make_shared(block_header, MySQLReplication::BinlogClientFactory::instance().getMetadata())}; +} + +} diff --git a/src/Storages/System/StorageSystemMySQLBinlogs.h b/src/Storages/System/StorageSystemMySQLBinlogs.h new file mode 100644 index 00000000000..a627137f495 --- /dev/null +++ b/src/Storages/System/StorageSystemMySQLBinlogs.h @@ -0,0 +1,29 @@ +#pragma once + +#include + +namespace DB +{ + +class StorageSystemMySQLBinlogs final : public IStorage +{ +public: + explicit StorageSystemMySQLBinlogs(const StorageID & storage_id_); + + std::string getName() const override { return "MySQLBinlogs"; } + + Pipe read( + 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; } + + static NamesAndTypesList getNamesAndTypes(); +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 01a41fc4b40..3ebe2a2d963 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -106,6 +106,10 @@ #include #endif +#if USE_MYSQL +#include +#endif + namespace DB { @@ -172,6 +176,9 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b #if USE_ROCKSDB attach(context, system_database, "rocksdb"); #endif +#if USE_MYSQL + attach(context, system_database, "mysql_binlogs"); +#endif attach(context, system_database, "parts"); attach(context, system_database, "projection_parts"); diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 286a2d29541..97c8b65f15d 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2714,3 +2714,698 @@ def table_with_indexes(clickhouse_node, mysql_node, service_name): mysql_node.query(f"DROP DATABASE IF EXISTS {db}") clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def binlog_client_test(clickhouse_node, mysql_node, replication): + db = "binlog_client_test" + replication.create_db_mysql(db) + + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100000, column="score") + replication.create_db_ch(f"{db}1", from_mysql_db=db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}1 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + + num_rows = replication.inserted_rows + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch(f"{db}2", from_mysql_db=db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}2 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + num_rows = replication.inserted_rows + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}1'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}2'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + # Catch up + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + + replication.drop_dbs_ch() + replication.create_db_ch( + f"{db}1", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.create_db_ch( + f"{db}2", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.insert_data(db, "t", 10000, column="score") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}1 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}2 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 10000, column="score") + + num_rows = replication.inserted_rows + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}1'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}2'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1", + ) + + mysql_node.query(f"UPDATE {db}.t SET score = score + 1") + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}2'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}3'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}1.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}2.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}3.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}1") + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=10, + ) + + +def binlog_client_timeout_test(clickhouse_node, mysql_node, replication): + db = "binlog_client_timeout_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 10000, column="score") + num_rows = replication.inserted_rows + + replication.create_db_ch( + f"{db}1", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=100000000, max_milliseconds_to_wait_in_binlog_queue=60000", + ) + replication.create_db_ch( + f"{db}2", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=100", + ) + replication.create_db_ch( + f"{db}4", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=10", + ) + + # After incremental sync + check_query( + clickhouse_node, + f"/* expect: 100000000, 60000 */ SELECT max_bytes, max_waiting_ms FROM system.mysql_binlogs WHERE name = '{db}1'", + f"100000000\t60000\n", + interval_seconds=1, + retry_count=10, + ) + check_query( + clickhouse_node, + f"/* expect: 10 */ SELECT max_bytes FROM system.mysql_binlogs WHERE name = '{db}2'", + f"10\n", + interval_seconds=2, + retry_count=10, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}4.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE {db}3") + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=10", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}3 FORMAT TSV", + "t\n", + ) + + clickhouse_node.query(f"DROP DATABASE {db}4") + replication.create_db_ch( + f"{db}4", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=50", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}4 FORMAT TSV", + "t\n", + ) + + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 1000 AND id < 100000" + ) + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 1000") + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}1.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}2.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}3.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}4.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}2'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}3'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}4'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}4.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}1.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}2.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}3.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}4.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + + +def wrong_password_test(clickhouse_node, mysql_node, replication): + db = "wrong_password_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100, column="score") + with pytest.raises(Exception) as exc: + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{replication.mysql_host}:3306', '{db}', 'root', 'wrong_password') SETTINGS use_binlog_client=1" + ) + + replication.create_db_ch(db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100, column="score") + check_query( + clickhouse_node, + f"/* expect: 200 */ SELECT COUNT() FROM {db}.t ", + "200\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: root@{replication.mysql_host}:3306 */ SELECT binlog_client_name FROM system.mysql_binlogs WHERE name = '{db}'", + f"root@{replication.mysql_host}:3306\n", + interval_seconds=1, + retry_count=30, + ) + + +def dispatcher_buffer_test(clickhouse_node, mysql_node, replication): + db = "dispatcher_buffer_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100, column="score") + rows_count = 100 + replication.create_db_ch( + db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=0, max_flush_milliseconds_in_binlog_dispatcher=0", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + rows_count += 100000 + + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE {db}") + replication.create_db_ch( + f"{db}", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=1000, max_flush_milliseconds_in_binlog_dispatcher=1000", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 10000, column="score") + rows_count += 10000 + + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 10000") + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-11-01'" + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-11-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch( + db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=100000000, max_flush_milliseconds_in_binlog_dispatcher=1000", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + rows_count += 100000 + + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 10000") + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-11-01'" + ) + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM system.mysql_binlogs WHERE name = '{db}' AND (dispatcher_events_read_per_sec > 0 OR dispatcher_bytes_read_per_sec > 0 OR dispatcher_events_flush_per_sec > 0 OR dispatcher_bytes_flush_per_sec > 0)", + f"1\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-11-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + +def gtid_after_attach_test(clickhouse_node, mysql_node, replication): + db = "gtid_after_attach_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + + db_count = 6 + for i in range(db_count): + replication.create_db_ch( + f"{db}{i}", + from_mysql_db=db, + settings="use_binlog_client=1", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}0 FORMAT TSV", + "t\n", + ) + for i in range(int(db_count / 2)): + clickhouse_node.query(f"DETACH DATABASE {db}{i}") + + mysql_node.query(f"USE {db}") + rows = 10000 + for i in range(100): + mysql_node.query(f"ALTER TABLE t ADD COLUMN (e{i} INT)") + replication.insert_data(db, "t", rows, column="score") + + clickhouse_node.restart_clickhouse(stop_start_wait_sec=120) + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=300, + ) diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index f3369e25d94..727188a4b86 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -1,3 +1,4 @@ +import os import time import pymysql.cursors import pytest @@ -142,6 +143,145 @@ def clickhouse_node(): yield node_db +class ReplicationHelper: + def __init__(self, clickhouse, mysql, mysql_host=None): + self.clickhouse = clickhouse + self.mysql = mysql + self.created_mysql_dbs = [] + self.created_clickhouse_dbs = [] + self.base_mysql_settings = os.getenv("TEST_BASE_MYSQL_SETTINGS", "") + self.base_ch_settings = os.getenv("TEST_BASE_CH_SETTINGS", "") + self.mysql_host = mysql_host if mysql_host is not None else cluster.mysql8_host + self.created_insert_procedures = {} + self.inserted_rows_per_sp = {} + self.inserted_rows = 0 + + def create_dbs(self, db_name, ch_settings="", mysql_settings=""): + self.create_db_mysql(db_name, settings=mysql_settings) + self.create_db_ch(db_name, settings=ch_settings) + + def create_db_mysql(self, db_name, settings=""): + self.mysql.query(f"DROP DATABASE IF EXISTS {db_name}") + self.mysql.query( + f"CREATE DATABASE {db_name} {self.base_mysql_settings} {settings}" + ) + self.created_mysql_dbs.append(db_name) + + def create_db_ch( + self, db_name, from_mysql_db=None, settings="", table_overrides="" + ): + if from_mysql_db is None: + from_mysql_db = db_name + self.clickhouse.query(f"DROP DATABASE IF EXISTS {db_name}") + all_settings = "" + create_query = f"CREATE DATABASE {db_name} ENGINE = MaterializedMySQL('{self.mysql_host}:3306', '{from_mysql_db}', 'root', 'clickhouse')" + if self.base_ch_settings or settings: + separator = ", " if self.base_ch_settings and settings else "" + create_query += f" SETTINGS {self.base_ch_settings}{separator}{settings}" + if table_overrides: + create_query += f" {table_overrides}" + self.clickhouse.query(create_query) + self.created_clickhouse_dbs.append(db_name) + + def drop_dbs_mysql(self): + for db_name in self.created_mysql_dbs: + self.mysql.query(f"DROP DATABASE IF EXISTS {db_name}") + self.created_mysql_dbs = [] + self.created_insert_procedures = {} + self.inserted_rows_per_sp = {} + self.inserted_rows = 0 + + def drop_dbs_ch(self): + for db_name in self.created_clickhouse_dbs: + self.clickhouse.query(f"DROP DATABASE IF EXISTS {db_name}") + self.created_clickhouse_dbs = [] + + def drop_dbs(self): + self.drop_dbs_mysql() + self.drop_dbs_ch() + + def create_stored_procedure(self, db, table, column): + sp_id = f"{db}_{table}_{column}" + if sp_id in self.created_insert_procedures: + return sp_id + self.mysql.query(f"DROP PROCEDURE IF EXISTS {db}.insert_test_data_{sp_id}") + self.mysql.query( + f""" +CREATE PROCEDURE {db}.insert_test_data_{sp_id}(IN num_rows INT, IN existing_rows INT) +BEGIN + DECLARE i INT; + SET i = existing_rows; + SET @insert = concat("INSERT INTO {table} ({column}) VALUES "); + SET @exedata = ""; + WHILE i < (num_rows + existing_rows) DO + SET @exedata=concat(@exedata, ",(", i , ")"); + SET i = i + 1; + IF i % 1000 = 0 + THEN + SET @exedata = SUBSTRING(@exedata, 2); + SET @exesql = concat(@insert, @exedata); + PREPARE stmt FROM @exesql; + EXECUTE stmt; + DEALLOCATE PREPARE stmt; + SET @exedata = ""; + END IF; + END WHILE; + IF length(@exedata) > 0 + THEN + SET @exedata = SUBSTRING(@exedata, 2); + SET @exesql = concat(@insert, @exedata); + PREPARE stmt FROM @exesql; + EXECUTE stmt; + DEALLOCATE PREPARE stmt; + END IF; +END""" + ) + self.created_insert_procedures[sp_id] = True + self.inserted_rows_per_sp[sp_id] = 0 + return sp_id + + def insert_data(self, db, table, num_rows, column="id"): + """Inserts num_rows into db.table, into the column `column` (which must be INT)""" + sp_id = self.create_stored_procedure(db, table, column) + self.mysql.query( + f"CALL {db}.insert_test_data_{sp_id}({num_rows}, {self.inserted_rows_per_sp[sp_id]})" + ) + self.inserted_rows_per_sp[sp_id] += num_rows + self.inserted_rows += num_rows + + def wait_for_sync_to_catch_up( + self, database: str = "", retry_count=30, interval_seconds=1 + ): + if database == "": + database = self.created_clickhouse_dbs[-1] + mysql_gtid = self.mysql.query_and_get_data("SELECT @@GLOBAL.gtid_executed")[0][ + 0 + ] + materialized_with_ddl.check_query( + self.clickhouse, + f"SELECT executed_gtid_set /* expect: {mysql_gtid} */ FROM system.materialized_mysql_databases WHERE name = '{database}'", + f"{mysql_gtid}\n", + retry_count=retry_count, + interval_seconds=interval_seconds, + ) + + +@pytest.fixture(scope="function") +def replication(started_mysql_8_0, request): + try: + replication = ReplicationHelper(node_db, started_mysql_8_0) + yield replication + finally: + if hasattr(request.session, "testsfailed") and request.session.testsfailed: + logging.warning(f"tests failed - not dropping databases") + else: + # drop databases only if the test succeeds - so we can inspect the database after failed tests + try: + replication.drop_dbs() + except Exception as e: + logging.warning(f"replication.drop_dbs() failed: {e}") + + def test_materialized_database_dml_with_mysql_5_7( started_cluster, started_mysql_5_7, clickhouse_node: ClickHouseInstance ): @@ -556,3 +696,21 @@ def test_table_with_indexes(started_cluster, started_mysql_8_0, clickhouse_node) materialized_with_ddl.table_with_indexes( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_binlog_client(started_cluster, started_mysql_8_0, replication): + materialized_with_ddl.binlog_client_test(node_db, started_mysql_8_0, replication) + replication.drop_dbs() + materialized_with_ddl.binlog_client_timeout_test( + node_db, started_mysql_8_0, replication + ) + replication.drop_dbs() + materialized_with_ddl.wrong_password_test(node_db, started_mysql_8_0, replication) + replication.drop_dbs() + materialized_with_ddl.dispatcher_buffer_test( + node_db, started_mysql_8_0, replication + ) + replication.drop_dbs() + materialized_with_ddl.gtid_after_attach_test( + node_db, started_mysql_8_0, replication + ) diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index d1f868eba97..484dd46a90c 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -1,173 +1,98 @@ +#include +#include +#include #include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -static DB::MySQLCharsetPtr charset = std::make_shared(); -static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( - DB::MySQLReplication::EventHeader & header, DB::ReadBuffer & payload, - std::shared_ptr & last_table_map_event, bool exist_checksum) +bool quit = false; +void signal_handler(int) { + quit = true; +} + +static void processBinlogFromFile(const std::string & bin_path, bool disable_checksum) +{ + DB::MySQLReplication::BinlogFromFile binlog; + binlog.open(bin_path); + binlog.setChecksum(disable_checksum ? DB::MySQLReplication::IBinlog::NONE : DB::MySQLReplication::IBinlog::CRC32); + DB::MySQLReplication::BinlogEventPtr event; - DB::ReadBufferPtr limit_read_buffer = std::make_shared(payload, header.event_size - 19, - /* trow_exception */ false, /* exact_limit */ std::nullopt); - DB::ReadBufferPtr event_payload = std::make_shared(*limit_read_buffer, exist_checksum ? 4 : 0); - - switch (header.type) + while (binlog.tryReadEvent(event, /*timeout*/ 0) && !quit) { - case DB::MySQLReplication::FORMAT_DESCRIPTION_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::ROTATE_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::QUERY_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - - auto query = std::static_pointer_cast(event); - switch (query->typ) - { - case DB::MySQLReplication::QUERY_EVENT_MULTI_TXN_FLAG: - case DB::MySQLReplication::QUERY_EVENT_XA: - { - event = std::make_shared(std::move(query->header)); - break; - } - default: - break; - } - break; - } - case DB::MySQLReplication::XID_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::TABLE_MAP_EVENT: - { - DB::MySQLReplication::TableMapEventHeader map_event_header; - map_event_header.parse(*event_payload); - event = std::make_shared(std::move(header), map_event_header, charset); - event->parseEvent(*event_payload); - last_table_map_event = std::static_pointer_cast(event); - break; - } - case DB::MySQLReplication::WRITE_ROWS_EVENT_V1: - case DB::MySQLReplication::WRITE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::DELETE_ROWS_EVENT_V1: - case DB::MySQLReplication::DELETE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::UPDATE_ROWS_EVENT_V1: - case DB::MySQLReplication::UPDATE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::GTID_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - default: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } + DB::WriteBufferFromOStream cout(std::cout); + event->dump(cout); + binlog.getPosition().dump(cout); + cout.finalize(); } - - return event; } -static int checkBinLogFile(const std::string & bin_path, bool exist_checksum) +static void processBinlogFromSocket(const std::string & host, int port, const std::string & user, const std::string & password, const std::string & executed_gtid_set, bool disable_checksum) { - DB::ReadBufferFromFile in(bin_path); - DB::assertString("\xfe\x62\x69\x6e", in); /// magic number + DB::MySQLReplication::BinlogFromSocket binlog; + binlog.setChecksum(disable_checksum ? DB::MySQLReplication::IBinlog::NONE : DB::MySQLReplication::IBinlog::CRC32); - DB::MySQLReplication::BinlogEventPtr last_event; - std::shared_ptr last_header; - std::shared_ptr table_map; + binlog.connect(host, port, user, password); + binlog.start(/*unique number*/ 42, executed_gtid_set); + DB::MySQLReplication::BinlogEventPtr event; - try + while (!quit) { - while (!in.eof()) + if (binlog.tryReadEvent(event, /*timeout*/ 100)) { - last_header = std::make_shared(); - last_header->parse(in); - last_event = parseSingleEventBody(*last_header, in, table_map, exist_checksum); + if (event->header.type != DB::MySQLReplication::HEARTBEAT_EVENT) + { + DB::WriteBufferFromOStream cout(std::cout); + event->dump(cout); + binlog.getPosition().dump(cout); + cout.finalize(); + } } } - catch (...) - { - DB::WriteBufferFromOStream cerr(std::cerr); - cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: " - << DB::getCurrentExceptionMessage(false) << '\n' << ", Previous event: " << '\n'; - last_event->dump(cerr); - cerr << '\n' << ", Event header: " << '\n'; - last_header->dump(cerr); - cerr << '\n'; - return DB::getCurrentExceptionCode(); - } - - DB::WriteBufferFromOStream cout(std::cout); - cout << "Check passed. " << '\n' << "No exception was thrown." << '\n' << "The last binlog event: " << '\n'; - last_event->dump(cout); - cout << '\n'; - return 0; } - int main(int argc, char ** argv) { + (void)signal(SIGINT, signal_handler); boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help,h", "Produce help message"); - desc.add_options()("disable_checksum", "Disable checksums in binlog files."); - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + std::string host = "127.0.0.1"; + int port = 3306; + std::string user = "root"; + std::string password; + std::string gtid; - if (options.count("help") || argc < 2) + desc.add_options() + ("help", "Produce help message") + ("disable_checksum", "Disable checksums in binlog files.") + ("binlog", boost::program_options::value(), "Binlog file") + ("host", boost::program_options::value(&host)->default_value(host), "Host to connect") + ("port", boost::program_options::value(&port)->default_value(port), "Port number to connect") + ("user", boost::program_options::value(&user)->default_value(user), "User") + ("password", boost::program_options::value(&password), "Password") + ("gtid", boost::program_options::value(>id), "Executed gtid set"); + + try { - std::cout << "Usage: " << argv[0] << " mysql_binlog_file" << std::endl; - std::cout << desc << std::endl; - return 1; + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + boost::program_options::notify(options); + + if (options.count("help") || (!options.count("binlog") && !options.count("gtid"))) + { + std::cout << "Usage: " << argv[0] << std::endl; + std::cout << desc << std::endl; + return EXIT_FAILURE; + } + + if (options.count("binlog")) + processBinlogFromFile(options["binlog"].as(), options.count("disable_checksum")); + else + processBinlogFromSocket(host, port, user, password, gtid, options.count("disable_checksum")); + } + catch (std::exception & ex) + { + std::cerr << ex.what() << std::endl; + return EXIT_FAILURE; } - return checkBinLogFile(argv[argc - 1], !options.count("disable_checksum")); + return EXIT_SUCCESS; } From 7dd128f90f2810ce640f841a05bd0733594ae1ec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jan 2024 16:00:50 +0000 Subject: [PATCH 050/116] Revert "remove projection from StorageSnapshot" This reverts commit a01acf5d2a650fa7e7d4a2a7426fbdc29c5142c5. --- .../Passes/ShardNumColumnToFunctionPass.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 6 +++--- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 2 +- .../optimizeUseAggregateProjection.cpp | 15 +++++++-------- .../Optimizations/optimizeUseNormalProjection.cpp | 4 +++- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 11 +++++++---- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageMerge.cpp | 8 ++++++++ src/Storages/StorageSnapshot.cpp | 10 +++++----- src/Storages/StorageSnapshot.h | 8 ++++++++ 12 files changed, 46 insertions(+), 26 deletions(-) diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp index f3ff7fe06b1..52c30b7b35d 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -46,7 +46,7 @@ public: return; const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - if (!storage->isVirtualColumn(column.name, storage_snapshot->metadata)) + if (!storage->isVirtualColumn(column.name, storage_snapshot->getMetadataForQuery())) return; auto function_node = std::make_shared("shardNum"); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 89b014a0360..9cbf24091e3 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1146,7 +1146,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select { for (const auto & name_type : storage_virtuals) { - if (name_type.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->metadata)) + if (name_type.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery())) { has_virtual_shard_num = true; break; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 2736f805271..56ac3f54b06 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -397,20 +397,20 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info /// We evaluate sampling for Merge lazily so we need to get all the columns if (storage_snapshot->storage.getName() == "Merge") { - const auto columns = storage_snapshot->metadata->getColumns().getAll(); + const auto columns = storage_snapshot->getMetadataForQuery()->getColumns().getAll(); for (const auto & column : columns) required_columns.insert(column.name); } else { - auto columns_required_for_sampling = storage_snapshot->metadata->getColumnsRequiredForSampling(); + auto columns_required_for_sampling = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForSampling(); required_columns.insert(columns_required_for_sampling.begin(), columns_required_for_sampling.end()); } } if (table_expression_modifiers->hasFinal()) { - auto columns_required_for_final = storage_snapshot->metadata->getColumnsRequiredForFinal(); + auto columns_required_for_final = storage_snapshot->getMetadataForQuery()->getColumnsRequiredForFinal(); required_columns.insert(columns_required_for_final.begin(), columns_required_for_final.end()); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index f7d31a54373..5c5171d4296 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -137,7 +137,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) { - const auto & sampling_key = storage_snapshot->metadata->getSamplingKey(); + const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes(); for (const auto & column : sampling_source_columns) required_columns_after_filter.push_back(ColumnWithTypeAndName(column.type, column.name)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f7d97f71d08..169ef13e732 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -669,7 +669,9 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & else { auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared(storage_snapshot->storage, best_candidate->projection->metadata); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); + proj_snapshot->addProjection(best_candidate->projection); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; @@ -694,19 +696,16 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique(std::move(pipe)); } - has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; - if (has_ordinary_parts) - reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); - projection_name = Context::QualifiedProjectionName { .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name, }; - } - if (!query_info.is_internal && context->hasQueryContext()) - context->getQueryContext()->addQueryAccessInfo(projection_name); + has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; + if (has_ordinary_parts) + reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); + } // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index f821e9c7b42..90de861651a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -196,7 +196,9 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared(storage_snapshot->storage, best_candidate->projection->metadata); + auto proj_snapshot = std::make_shared( + storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + proj_snapshot->addProjection(best_candidate->projection); auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d446ca17634..9f133ff075f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -261,7 +261,7 @@ ReadFromMergeTree::ReadFromMergeTree( , prewhere_info(query_info_.prewhere_info) , actions_settings(ExpressionActionsSettings::fromContext(context_)) , storage_snapshot(std::move(storage_snapshot_)) - , metadata_for_reading(storage_snapshot->metadata) + , metadata_for_reading(storage_snapshot->getMetadataForQuery()) , context(std::move(context_)) , block_size{ .max_block_size_rows = max_block_size_, @@ -311,7 +311,7 @@ ReadFromMergeTree::ReadFromMergeTree( updateSortDescriptionForOutputStream( *output_stream, - storage_snapshot->metadata->getSortingKeyColumns(), + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), query_info.input_order_info, prewhere_info); @@ -1695,7 +1695,7 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info updateSortDescriptionForOutputStream( *output_stream, - storage_snapshot->metadata->getSortingKeyColumns(), + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), query_info.input_order_info, prewhere_info); @@ -1896,8 +1896,11 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons partition_names.emplace_back( fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); } - context->getQueryContext()->addQueryAccessInfo(partition_names); + + if (storage_snapshot->projection) + context->getQueryContext()->addQueryAccessInfo( + Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index e0a0a3313fe..f5f0fa6f726 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -70,7 +70,7 @@ bool injectRequiredColumnsRecursively( /// Column doesn't have default value and don't exist in part /// don't need to add to required set. - auto metadata_snapshot = storage_snapshot->metadata; + auto metadata_snapshot = storage_snapshot->getMetadataForQuery(); const auto column_default = metadata_snapshot->getColumns().getDefault(column_name); if (!column_default) return false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3546e3c0e88..2dec0f8257e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6896,7 +6896,7 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead( auto result_ptr = reader.estimateNumMarksToRead( parts, query_info.prewhere_info, - storage_snapshot->metadata->getColumns().getAll().getNames(), + storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), storage_snapshot->metadata, storage_snapshot->metadata, query_info, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4b89dc5fc7c..a03ad365767 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -545,6 +545,14 @@ void ReadFromMerge::createChildPlans() query_info.input_order_info = input_sorting_info; } +<<<<<<< HEAD +======= + auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); + + std::vector> pipelines; + QueryPlanResourceHolder resources; + +>>>>>>> parent of a01acf5d2a6 (remove projection from StorageSnapshot) for (const auto & table : selected_tables) { size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index af9d5fd1b34..ada3252630f 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -38,7 +38,7 @@ void StorageSnapshot::init() NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const { - auto all_columns = metadata->getColumns().get(options); + auto all_columns = getMetadataForQuery()->getColumns().get(options); if (options.with_extended_objects) extendObjectColumns(all_columns, object_columns, options.with_subcolumns); @@ -85,7 +85,7 @@ NamesAndTypesList StorageSnapshot::getColumnsByNames(const GetColumnsOptions & o std::optional StorageSnapshot::tryGetColumn(const GetColumnsOptions & options, const String & column_name) const { - const auto & columns = metadata->getColumns(); + const auto & columns = getMetadataForQuery()->getColumns(); auto column = columns.tryGetColumn(options, column_name); if (column && (!column->type->hasDynamicSubcolumns() || !options.with_extended_objects)) return column; @@ -127,7 +127,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons { Block res; - const auto & columns = metadata->getColumns(); + const auto & columns = getMetadataForQuery()->getColumns(); for (const auto & column_name : column_names) { auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); @@ -159,7 +159,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons ColumnsDescription StorageSnapshot::getDescriptionForColumns(const Names & column_names) const { ColumnsDescription res; - const auto & columns = metadata->getColumns(); + const auto & columns = getMetadataForQuery()->getColumns(); for (const auto & name : column_names) { auto column = columns.tryGetColumnOrSubcolumnDescription(GetColumnsOptions::All, name); @@ -196,7 +196,7 @@ namespace void StorageSnapshot::check(const Names & column_names) const { - const auto & columns = metadata->getColumns(); + const auto & columns = getMetadataForQuery()->getColumns(); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(); if (column_names.empty()) diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 9856164bc9f..d62e118e1f2 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -25,6 +25,9 @@ struct StorageSnapshot using DataPtr = std::unique_ptr; DataPtr data; + /// Projection that is used in query. + mutable const ProjectionDescription * projection = nullptr; + StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_) @@ -80,6 +83,11 @@ struct StorageSnapshot DataTypePtr getConcreteType(const String & column_name) const; + void addProjection(const ProjectionDescription * projection_) const { projection = projection_; } + + /// If we have a projection then we should use its metadata. + StorageMetadataPtr getMetadataForQuery() const { return projection ? projection->metadata : metadata; } + private: void init(); From cf4604bfb69c7450e79f9f791a257fd3fa8c3c0f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jan 2024 16:53:47 +0000 Subject: [PATCH 051/116] fix build and tests --- .../Optimizations/optimizeUseAggregateProjection.cpp | 3 +-- .../Optimizations/optimizeUseNormalProjection.cpp | 3 +-- src/Storages/StorageMerge.cpp | 10 +--------- src/Storages/StorageSnapshot.cpp | 3 +++ 4 files changed, 6 insertions(+), 13 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 169ef13e732..198bdbbb71d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -669,8 +669,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & else { auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); + auto proj_snapshot = std::make_shared(storage_snapshot->storage, storage_snapshot->metadata); proj_snapshot->addProjection(best_candidate->projection); auto query_info_copy = query_info; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 90de861651a..05afc80cba0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -196,8 +196,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } auto storage_snapshot = reading->getStorageSnapshot(); - auto proj_snapshot = std::make_shared( - storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); + auto proj_snapshot = std::make_shared(storage_snapshot->storage, storage_snapshot->metadata); proj_snapshot->addProjection(best_candidate->projection); auto query_info_copy = query_info; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e0ef2110606..0d67403fa2f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -488,14 +488,6 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ query_info_.input_order_info = input_sorting_info; } -<<<<<<< HEAD -======= - auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); - - std::vector> pipelines; - QueryPlanResourceHolder resources; - ->>>>>>> parent of a01acf5d2a6 (remove projection from StorageSnapshot) for (const auto & table : selected_tables) { size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); @@ -548,7 +540,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ ASTPtr required_columns_expr_list = std::make_shared(); ASTPtr column_expr; - auto sample_block = merge_storage_snapshot->metadata->getSampleBlock(); + auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); for (const auto & column : real_column_names) { diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index ada3252630f..34c092c7208 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -21,7 +21,10 @@ namespace ErrorCodes std::shared_ptr StorageSnapshot::clone(DataPtr data_) const { auto res = std::make_shared(storage, metadata, object_columns); + + res->projection = projection; res->data = std::move(data_); + return res; } From 48a3968d5a7d955c3acbbecea963d2b60a20f076 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jan 2024 20:06:36 +0000 Subject: [PATCH 052/116] fix query info with projection --- .../Optimizations/optimizeUseAggregateProjection.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 198bdbbb71d..f0cf52aa46d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -706,6 +706,15 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); } + if (!query_info.is_internal && context->hasQueryContext()) + { + context->getQueryContext()->addQueryAccessInfo(Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); + } + // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); From 2e14cfb526870955e849da754a42891b82f02c6a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 03:40:26 +0000 Subject: [PATCH 053/116] add settings for output compression level and window size Signed-off-by: Duc Canh Le --- src/Core/Settings.h | 2 + src/IO/CompressionMethod.cpp | 4 +- src/IO/CompressionMethod.h | 1 + src/IO/ZstdDeflatingWriteBuffer.cpp | 42 ++++++++++++++----- src/IO/ZstdDeflatingWriteBuffer.h | 1 + src/Interpreters/executeQuery.cpp | 5 ++- src/Processors/Formats/Impl/Parquet/Write.cpp | 1 + .../WriteBufferFromHTTPServerResponse.cpp | 1 + src/Storages/HDFS/StorageHDFS.cpp | 10 ++--- src/Storages/StorageAzureBlob.cpp | 7 +++- src/Storages/StorageFile.cpp | 8 +++- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 5 ++- 13 files changed, 65 insertions(+), 26 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4e057861f60..680b80d92f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -204,6 +204,8 @@ class IColumn; M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \ + M(UInt64, output_format_compression_level, 3, "Default compression level if output is compressed", 0) \ + M(UInt64, output_format_compression_zstd_window_log, 0, "If > 0, explicitly set window size and enable long-range mode for zstd compression, expressed as power of 2", 0) \ \ M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 13e1adbb702..fc415b73ec1 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -170,7 +170,7 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( } std::unique_ptr wrapWriteBufferWithCompressionMethod( - std::unique_ptr nested, CompressionMethod method, int level, size_t buf_size, char * existing_memory, size_t alignment) + std::unique_ptr nested, CompressionMethod method, int level, int zstd_window_log, size_t buf_size, char * existing_memory, size_t alignment) { if (method == DB::CompressionMethod::Gzip || method == CompressionMethod::Zlib) return std::make_unique(std::move(nested), method, level, buf_size, existing_memory, alignment); @@ -183,7 +183,7 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); if (method == CompressionMethod::Zstd) - return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); + return std::make_unique(std::move(nested), level, zstd_window_log, buf_size, existing_memory, alignment); if (method == CompressionMethod::Lz4) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index c142531cd05..511704059ec 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -66,6 +66,7 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( std::unique_ptr nested, CompressionMethod method, int level, + int zstd_window_log = 0, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 949d65926b3..20bdeef7918 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -6,25 +6,45 @@ namespace DB namespace ErrorCodes { extern const int ZSTD_ENCODER_FAILED; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +static void setZstdParameter(ZSTD_CCtx * cctx, ZSTD_cParameter param, int value) +{ + auto ret = ZSTD_CCtx_setParameter(cctx, param, value); + if (ZSTD_isError(ret)) + throw Exception( + ErrorCodes::ZSTD_ENCODER_FAILED, + "zstd stream encoder option setting failed: error code: {}; zstd version: {}", + ret, + ZSTD_VERSION_STRING); } ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( - std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) + std::unique_ptr out_, int compression_level, int window_log, size_t buf_size, char * existing_memory, size_t alignment) : WriteBufferWithOwnMemoryDecorator(std::move(out_), buf_size, existing_memory, alignment) { cctx = ZSTD_createCCtx(); if (cctx == nullptr) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder init failed: zstd version: {}", ZSTD_VERSION_STRING); - size_t ret = ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, compression_level); - if (ZSTD_isError(ret)) - throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, - "zstd stream encoder option setting failed: error code: {}; zstd version: {}", - ret, ZSTD_VERSION_STRING); - ret = ZSTD_CCtx_setParameter(cctx, ZSTD_c_checksumFlag, 1); - if (ZSTD_isError(ret)) - throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, - "zstd stream encoder option setting failed: error code: {}; zstd version: {}", - ret, ZSTD_VERSION_STRING); + setZstdParameter(cctx, ZSTD_c_compressionLevel, compression_level); + + if (window_log > 0) + { + ZSTD_bounds window_log_bounds = ZSTD_cParam_getBounds(ZSTD_c_windowLog); + if (ZSTD_isError(window_log_bounds.error)) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTD windowLog parameter is not supported {}", + std::string(ZSTD_getErrorName(window_log_bounds.error))); + if (window_log > window_log_bounds.upperBound || window_log < window_log_bounds.lowerBound) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, + "ZSTD codec can't have window log more than {} and lower than {}, given {}", + toString(window_log_bounds.upperBound), + toString(window_log_bounds.lowerBound), toString(window_log)); + setZstdParameter(cctx, ZSTD_c_enableLongDistanceMatching, 1); + setZstdParameter(cctx, ZSTD_c_windowLog, window_log); + } + + setZstdParameter(cctx, ZSTD_c_checksumFlag, 1); input = {nullptr, 0, 0}; output = {nullptr, 0, 0}; diff --git a/src/IO/ZstdDeflatingWriteBuffer.h b/src/IO/ZstdDeflatingWriteBuffer.h index a66d6085a74..8c129b1bfbb 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.h +++ b/src/IO/ZstdDeflatingWriteBuffer.h @@ -17,6 +17,7 @@ public: ZstdDeflatingWriteBuffer( std::unique_ptr out_, int compression_level, + int window_log = 0, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 63804d2d86f..23a1a703b16 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1434,11 +1434,12 @@ void executeQuery( const auto & compression_method_node = ast_query_with_output->compression->as(); compression_method = compression_method_node.value.safeGet(); } - + const auto & settings = context->getSettingsRef(); compressed_buffer = wrapWriteBufferWithCompressionMethod( std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), chooseCompressionMethod(out_file, compression_method), - /* compression level = */ 3 + /* compression level = */ static_cast(settings.output_format_compression_level), + /* zstd_window_log = */ static_cast(settings.output_format_compression_zstd_window_log) ); } diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 6d8f1ab55cb..02ca2734ff8 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -448,6 +448,7 @@ PODArray & compress(PODArray & source, PODArray & scratch, Com std::move(dest_buf), method, /*level*/ 3, + /*zstd_window_log*/ 0, source.size(), /*existing_memory*/ source.data()); chassert(compressed_buf->position() == source.data()); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 1a12c09a8c7..c32da278e49 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -112,6 +112,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() std::make_unique(*response_body_ostr), compress ? compression_method : CompressionMethod::None, compression_level, + 0, working_buffer.size(), working_buffer.begin()); else diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index fdbb5e9f171..a22c5903420 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -723,13 +723,13 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { + const auto & settings = context->getSettingsRef(); write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( - uri, - context->getGlobalContext()->getConfigRef(), - context->getSettingsRef().hdfs_replication, - context->getWriteSettings()), - compression_method, 3); + uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, context->getWriteSettings()), + compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 9564bad485c..0e00c62111c 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -533,7 +533,12 @@ public: , format_settings(format_settings_) { StoredObject object(blob_path); - write_buf = wrapWriteBufferWithCompressionMethod(object_storage->writeObject(object, WriteMode::Rewrite), compression_method, 3); + const auto & settings = context->getSettingsRef(); + write_buf = wrapWriteBufferWithCompressionMethod( + object_storage->writeObject(object, WriteMode::Rewrite), + compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 25bb6691ff6..0c839406462 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1485,8 +1485,12 @@ public: /// In case of formats with prefixes if file is not empty we have already written prefix. bool do_not_write_prefix = naked_buffer->size(); - - write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); + const auto & settings = context->getSettingsRef(); + write_buf = wrapWriteBufferWithCompressionMethod( + std::move(naked_buffer), + compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name, *write_buf, metadata_snapshot->getSampleBlock(), context, format_settings); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 60ae7f219f4..1af0b638df9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -932,6 +932,7 @@ public: blob_log->query_id = context->getCurrentQueryId(); } + const auto & settings = context->getSettingsRef(); write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( configuration_.client, @@ -944,7 +945,8 @@ public: threadPoolCallbackRunner(getIOThreadPool().get(), "S3ParallelWrite"), context->getWriteSettings()), compression_method, - 3); + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d6b6f5af61c..39fd6195698 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -540,11 +540,12 @@ StorageURLSink::StorageURLSink( Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config ); + const auto & settings = context->getSettingsRef(); write_buf = wrapWriteBufferWithCompressionMethod( std::move(write_buffer), compression_method, - 3 - ); + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings); } From 03951e1b21469e747f2a894aff5365c8b30565eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Jan 2024 21:59:32 +0100 Subject: [PATCH 054/116] Make a test not depend on the lack of floating point associativity --- .../01603_decimal_mult_float.reference | 24 ++++++++-------- .../0_stateless/01603_decimal_mult_float.sql | 28 +++++++++---------- 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.reference b/tests/queries/0_stateless/01603_decimal_mult_float.reference index 4c9d45423ee..72b10d768f1 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.reference +++ b/tests/queries/0_stateless/01603_decimal_mult_float.reference @@ -1,14 +1,14 @@ 2.4 10.165 -0.00012000000000000002 -150.16500000000002 -7.775900000000001 -56.622689999999984 -598.8376688440277 -299.41883695311844 -0.7485470860550345 -2.2456412771483882 -1.641386318314034 -1.641386318314034 -1.6413863258732018 -1.6413863258732018 +0.00012 +150.165 +7.7759 +56.62269 +598.837669 +299.418837 +0.748547 +2.245641 +1.641386 +1.641386 +1.641386 +1.641386 diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.sql b/tests/queries/0_stateless/01603_decimal_mult_float.sql index 799ab91d332..1a4652df23a 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.sql +++ b/tests/queries/0_stateless/01603_decimal_mult_float.sql @@ -1,9 +1,9 @@ SET optimize_arithmetic_operations_in_aggregate_functions = 0; -SELECT toDecimal32(2, 2) * 1.2; -SELECT toDecimal64(0.5, 2) * 20.33; -SELECT 0.00001 * toDecimal32(12, 2); -SELECT 30.033 * toDecimal32(5, 1); +SELECT round(toDecimal32(2, 2) * 1.2, 6); +SELECT round(toDecimal64(0.5, 2) * 20.33, 6); +SELECT round(0.00001 * toDecimal32(12, 2), 6); +SELECT round(30.033 * toDecimal32(5, 1), 6); CREATE TABLE IF NOT EXISTS test01603 ( f64 Float64, @@ -13,17 +13,17 @@ CREATE TABLE IF NOT EXISTS test01603 ( INSERT INTO test01603(f64) SELECT 1 / (number + 1) FROM system.numbers LIMIT 1000; -SELECT sum(d * 1.1) FROM test01603; -SELECT sum(8.01 * d) FROM test01603; +SELECT round(sum(d * 1.1), 6) FROM test01603; +SELECT round(sum(8.01 * d), 6) FROM test01603; -SELECT sum(f64 * toDecimal64(80, 2)) FROM test01603; -SELECT sum(toDecimal64(40, 2) * f32) FROM test01603; -SELECT sum(f64 * toDecimal64(0.1, 2)) FROM test01603; -SELECT sum(toDecimal64(0.3, 2) * f32) FROM test01603; +SELECT round(sum(f64 * toDecimal64(80, 2)), 6) FROM test01603; +SELECT round(sum(toDecimal64(40, 2) * f32), 6) FROM test01603; +SELECT round(sum(f64 * toDecimal64(0.1, 2)), 6) FROM test01603; +SELECT round(sum(toDecimal64(0.3, 2) * f32), 6) FROM test01603; -SELECT sum(f64 * d) FROM test01603; -SELECT sum(d * f64) FROM test01603; -SELECT sum(f32 * d) FROM test01603; -SELECT sum(d * f32) FROM test01603; +SELECT round(sum(f64 * d), 6) FROM test01603; +SELECT round(sum(d * f64), 6) FROM test01603; +SELECT round(sum(f32 * d), 6) FROM test01603; +SELECT round(sum(d * f32), 6) FROM test01603; DROP TABLE IF EXISTS test01603; From 4b26a08318b44e7367262f200017a05749480b42 Mon Sep 17 00:00:00 2001 From: chenwei Date: Wed, 3 Jan 2024 15:38:55 +0800 Subject: [PATCH 055/116] keeper connection mode in doc file not work. 1. use "-h localhost:9181" followed doc is aborted. root@c64618ba15d1:~/data/zk# clickhouse-keeper-client -h localhost:9181 getservbyname Aborted 2. use "-h localhost -p 9181" is ok. root@c64618ba15d1:~/data/zk# clickhouse-keeper-client -h localhost -p 9181 Connected to ZooKeeper at 127.0.0.1:9181 with session_id 20 Keeper feature flag FILTERED_LIST: enabled Keeper feature flag MULTI_READ: enabled Keeper feature flag CHECK_NOT_EXISTS: disabled Keeper feature flag CREATE_IF_NOT_EXISTS: disabled / :) --- docs/en/operations/utilities/clickhouse-keeper-client.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index d6e11fb9613..4588f68cacd 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -24,7 +24,7 @@ A client application to interact with clickhouse-keeper by its native protocol. ## Example {#clickhouse-keeper-client-example} ```bash -./clickhouse-keeper-client -h localhost:9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 +./clickhouse-keeper-client -h localhost -p 9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 Connected to ZooKeeper at [::1]:9181 with session_id 137 / :) ls keeper foo bar From 29c4d96b48ce390781e3db3df5d3804292ecd14b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 5 Jan 2024 06:54:56 +0000 Subject: [PATCH 056/116] fix build and add a tests Signed-off-by: Duc Canh Le --- src/IO/ZstdDeflatingWriteBuffer.cpp | 1 + .../0_stateless/02961_output_format_compress_params.reference | 1 + .../0_stateless/02961_output_format_compress_params.sql | 4 ++++ 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02961_output_format_compress_params.reference create mode 100755 tests/queries/0_stateless/02961_output_format_compress_params.sql diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 20bdeef7918..3b474a4de74 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.reference b/tests/queries/0_stateless/02961_output_format_compress_params.reference new file mode 100644 index 00000000000..749fce669df --- /dev/null +++ b/tests/queries/0_stateless/02961_output_format_compress_params.reference @@ -0,0 +1 @@ +1000000 diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.sql b/tests/queries/0_stateless/02961_output_format_compress_params.sql new file mode 100755 index 00000000000..fc67bca5816 --- /dev/null +++ b/tests/queries/0_stateless/02961_output_format_compress_params.sql @@ -0,0 +1,4 @@ +INSERT INTO FUNCTION file('test_02961.csv', 'CSV', 'x UInt64', 'zstd') SELECT number FROM numbers(1000000) SETTINGS output_format_compression_level = 10, output_format_compression_zstd_window_log = 30, engine_file_truncate_on_insert = 1; +-- Simple check that output_format_compression_zstd_window_log = 30 works +SELECT count() FROM file('test_02961.csv', 'CSV', 'x UInt64', 'zstd'); -- { serverError ZSTD_DECODER_FAILED } +SELECT count() FROM file('test_02961.csv', 'CSV', 'x UInt64', 'zstd') SETTINGS zstd_window_log_max = 30; From 934e00621315218e602d3a01b53fbd8070edc40c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 5 Jan 2024 07:13:46 +0000 Subject: [PATCH 057/116] add document Signed-off-by: Duc Canh Le --- docs/en/operations/settings/settings.md | 18 ++++++++++++++++++ src/Core/Settings.h | 2 +- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6e087467bb9..1028edfffd0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4771,6 +4771,24 @@ Type: Int64 Default: 0 + +## output_format_compression_level + +Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`. + +Possible values: from `1` to `22` + +Default: `3` + + +## output_format_compression_zstd_window_log + +To be used when `output_format_compression_level` is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression. This can help to achive better compression ratio. + +Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`). + +Default: `0` + ## rewrite_count_distinct_if_with_count_distinct_implementation Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#count_distinct_implementation) setting. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 680b80d92f7..db0ffebe2a7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -204,7 +204,7 @@ class IColumn; M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \ - M(UInt64, output_format_compression_level, 3, "Default compression level if output is compressed", 0) \ + M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`", 0) \ M(UInt64, output_format_compression_zstd_window_log, 0, "If > 0, explicitly set window size and enable long-range mode for zstd compression, expressed as power of 2", 0) \ \ M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ From c3586b2e35c31b9cbc81d33b80eabec0bd4a080d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 5 Jan 2024 07:23:22 +0000 Subject: [PATCH 058/116] fix docs Signed-off-by: Duc Canh Le --- docs/en/operations/settings/settings.md | 4 ++-- src/Core/Settings.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 1028edfffd0..0cf6dbd80a1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4774,7 +4774,7 @@ Default: 0 ## output_format_compression_level -Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`. +Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when writing to table functions `file`, `url`, `hdfs`, `s3`, or `azureBlobStorage`. Possible values: from `1` to `22` @@ -4783,7 +4783,7 @@ Default: `3` ## output_format_compression_zstd_window_log -To be used when `output_format_compression_level` is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression. This can help to achive better compression ratio. +To be used when output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression. This can help to achieve better compression ratio. Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index db0ffebe2a7..254f7338a30 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -204,8 +204,8 @@ class IColumn; M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \ - M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`", 0) \ - M(UInt64, output_format_compression_zstd_window_log, 0, "If > 0, explicitly set window size and enable long-range mode for zstd compression, expressed as power of 2", 0) \ + M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`.", 0) \ + M(UInt64, output_format_compression_zstd_window_log, 0, "To be used when output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression.", 0) \ \ M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ From 53f36f8b2104073ad19ccee9bdb2613d7cef16a2 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 7 Nov 2023 13:35:46 +0100 Subject: [PATCH 059/116] object storage key template --- src/Common/MatchGenerator.cpp | 418 ++++++++++++++++++ src/Common/MatchGenerator.h | 31 ++ src/Common/ObjectStorageKey.cpp | 1 + src/Common/ObjectStorageKeyGenerator.cpp | 97 ++++ src/Common/ObjectStorageKeyGenerator.h | 22 + .../tests/gtest_generate_random_by_regexp.cpp | 97 ++++ .../DiskObjectStorageMetadata.cpp | 4 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 22 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 13 +- .../ObjectStorages/S3/registerDiskS3.cpp | 75 +++- .../configs/setting.xml | 11 + .../configs/storage_conf.xml | 3 +- .../configs/storage_conf_new.xml | 61 +++ .../test_backward_compatibility.py | 33 +- 14 files changed, 845 insertions(+), 43 deletions(-) create mode 100644 src/Common/MatchGenerator.cpp create mode 100644 src/Common/MatchGenerator.h create mode 100644 src/Common/ObjectStorageKeyGenerator.cpp create mode 100644 src/Common/ObjectStorageKeyGenerator.h create mode 100644 src/Common/tests/gtest_generate_random_by_regexp.cpp create mode 100644 tests/integration/test_remote_blobs_naming/configs/setting.xml create mode 100644 tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp new file mode 100644 index 00000000000..5fa644c20f9 --- /dev/null +++ b/src/Common/MatchGenerator.cpp @@ -0,0 +1,418 @@ +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +# pragma clang diagnostic ignored "-Wgnu-anonymous-struct" +# pragma clang diagnostic ignored "-Wnested-anon-types" +# pragma clang diagnostic ignored "-Wunused-parameter" +# pragma clang diagnostic ignored "-Wshadow-field-in-constructor" +# pragma clang diagnostic ignored "-Wdtor-name" +#endif +#include +#include +#include +#ifdef __clang__ +# pragma clang diagnostic pop +#endif + +#ifdef LOG_INFO +#undef LOG_INFO +#undef LOG_WARNING +#undef LOG_ERROR +#undef LOG_FATAL +#endif + +#include "MatchGenerator.h" + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +} +} + + +namespace re2 +{ + +class RandomStringPrepareWalker : public Regexp::Walker +{ +private: + static constexpr int ImplicitMax = 100; + + using Children = std::vector; + + class Generators; + + /// This function objects look much prettier than lambda expression when stack traces are printed + class NodeFunction + { + public: + virtual String operator() (const Generators& generators) = 0; + virtual ~NodeFunction() = default; + }; + + using NodeFunctionPtr = std::shared_ptr; + + class Generators: public std::map {}; + + class RegexpConcatFunction : public NodeFunction + { + public: + RegexpConcatFunction(Children children_) + : children(std::move(children_)) + {} + + String operator () (const Generators& generators_) override + { + String result; + for (auto child: children) + { + auto res = generators_.at(child)->operator()(generators_); + result.append(res); + } + return result; + } + + private: + Children children; + }; + + class RegexpAlternateFunction : public NodeFunction + { + public: + RegexpAlternateFunction(Children children_) + : children(std::move(children_)) + {} + + String operator () (const Generators& generators_) override + { + std::uniform_int_distribution distribution(0, static_cast(children.size()-1)); + int chosen = distribution(thread_local_rng); + return generators_.at(children[chosen])->operator()(generators_); + } + + private: + Children children; + }; + + class RegexpRepeatFunction : public NodeFunction + { + public: + RegexpRepeatFunction(Regexp * re_, int min_repeat_, int max_repeat_) + : re(re_) + , min_repeat(min_repeat_) + , max_repeat(max_repeat_) + {} + + String operator () (const Generators& generators_) override + { + std::uniform_int_distribution distribution(min_repeat, max_repeat); + int chosen = distribution(thread_local_rng); + + String result; + for (int i = 0; i < chosen; ++i) + result.append(generators_.at(re)->operator()(generators_)); + return result; + } + + private: + Regexp * re; + int min_repeat = 0; + int max_repeat = 0; + }; + + class RegexpCharClassFunction : public NodeFunction + { + public: + RegexpCharClassFunction(Regexp * re_) + : re(re_) + {} + + String operator () (const Generators&) override + { + CharClass * cc = re->cc(); + if (cc->empty()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "kRegexpCharClass is empty"); + + std::uniform_int_distribution distribution(1, cc->size()); + int chosen = distribution(thread_local_rng); + int count_down = chosen; + + auto it = cc->begin(); + for (; it != cc->end(); ++it) + { + auto range_len = it->hi - it->lo + 1; + if (count_down <= range_len) + break; + count_down -= range_len; + } + + if (it == cc->end()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, + "Unable to choose the rune. Runes {}, chosen {}", + cc->size(), chosen); + + Rune r = it->lo + count_down - 1; + char buffer[UTFmax+1]; + buffer[re2::runetochar(buffer, &r)] = 0; + return String(buffer); + } + + private: + Regexp * re; + }; + + class RegexpLiteralStringFunction : public NodeFunction + { + public: + RegexpLiteralStringFunction(Regexp * re_) + : re(re_) + {} + + String operator () (const Generators&) override + { + if (re->nrunes() == 0) + return String(); + + String result; + char buffer[UTFmax+1]; + for (int i = 0; i < re->nrunes(); ++i) + { + buffer[re2::runetochar(buffer, &re->runes()[i])] = 0; + result.append(buffer); + } + return result; + } + + private: + Regexp * re; + }; + + class RegexpLiteralFunction : public NodeFunction + { + public: + RegexpLiteralFunction(Regexp * re_) + : re(re_) + {} + + String operator () (const Generators&) override + { + String result; + char buffer[UTFmax+1]; + + Rune r = re->rune(); + buffer[re2::runetochar(buffer, &r)] = 0; + result.append(buffer); + + return result; + } + + private: + Regexp * re; + }; + + class ThrowExceptionFunction : public NodeFunction + { + public: + ThrowExceptionFunction(Regexp * re_) + : re(re_) + {} + + String operator () (const Generators&) override + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "RandomStringPrepareWalker: regexp node '{}' is not supported for generating a random match", + magic_enum::enum_name(re->op())); + } + + private: + Regexp * re; + }; + + +public: + RandomStringPrepareWalker(bool logging) + : logger(logging ? &Poco::Logger::get("GeneratorCombiner") : nullptr) + { + if (logger) + LOG_DEBUG(logger, "GeneratorCombiner"); + } + + std::function getGenerator() + { + if (root == nullptr) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no root has been set"); + + if (generators.size() == 0) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no generators"); + + auto result = [generators_ = std::move(generators), root_ = std::move(root)] () -> String { + return generators_.at(root_)->operator()(generators_); + }; + + root = nullptr; + generators.clear(); + + return std::move(result); + } + +private: + Children CopyChildrenArgs(Regexp** children, int nchild) + { + Children result; + result.reserve(nchild); + for (int i = 0; i < nchild; ++i) + result.push_back(Copy(children[i])); + return result; + } + + Regexp * ShortVisit(Regexp* /*re*/, Regexp * /*parent_arg*/) override + { + if (logger) + LOG_DEBUG(logger, "ShortVisit"); + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "should not be call"); + } + + Regexp * PreVisit(Regexp * re, Regexp* parent_arg, bool* /*stop*/) override /*noexcept*/ + { + if (logger) + LOG_DEBUG(logger, "GeneratorCombiner PreVisit node {}", magic_enum::enum_name(re->op())); + + if (parent_arg == nullptr) + { + chassert(root == nullptr); + chassert(re != nullptr); + root = re; + } + + return re; + } + + Regexp * PostVisit(Regexp * re, Regexp* /*parent_arg*/, Regexp* pre_arg, + Regexp ** child_args, int nchild_args) override /*noexcept*/ + { + if (logger) + LOG_DEBUG(logger, "GeneratorCombiner PostVisit node {}", + magic_enum::enum_name(re->op())); + + switch (re->op()) + { + case kRegexpConcat: // Matches concatenation of sub_[0..nsub-1]. + generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args)); + break; + case kRegexpAlternate: // Matches union of sub_[0..nsub-1]. + generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args)); + break; + case kRegexpQuest: // Matches sub_[0] zero or one times. + chassert(nchild_args == 1); + generators[re] = std::make_shared(child_args[0], 0, 1); + break; + case kRegexpStar: // Matches sub_[0] zero or more times. + chassert(nchild_args == 1); + generators[re] = std::make_shared(child_args[0], 0, ImplicitMax); + break; + case kRegexpPlus: // Matches sub_[0] one or more times. + chassert(nchild_args == 1); + generators[re] = std::make_shared(child_args[0], 1, ImplicitMax); + break; + case kRegexpCharClass: // Matches character class given by cc_. + chassert(nchild_args == 0); + generators[re] = std::make_shared(re); + break; + case kRegexpLiteralString: // Matches runes_. + chassert(nchild_args == 0); + generators[re] = std::make_shared(re); + break; + case kRegexpLiteral: // Matches rune_. + chassert(nchild_args == 0); + generators[re] = std::make_shared(re); + break; + case kRegexpCapture: // Parenthesized (capturing) subexpression. + chassert(nchild_args == 1); + generators[re] = generators[child_args[0]]; + break; + + case kRegexpNoMatch: // Matches no strings. + case kRegexpEmptyMatch: // Matches empty string. + case kRegexpRepeat: // Matches sub_[0] at least min_ times, at most max_ times. + case kRegexpAnyChar: // Matches any character. + case kRegexpAnyByte: // Matches any byte [sic]. + case kRegexpBeginLine: // Matches empty string at beginning of line. + case kRegexpEndLine: // Matches empty string at end of line. + case kRegexpWordBoundary: // Matches word boundary "\b". + case kRegexpNoWordBoundary: // Matches not-a-word boundary "\B". + case kRegexpBeginText: // Matches empty string at beginning of text. + case kRegexpEndText: // Matches empty string at end of text. + case kRegexpHaveMatch: // Forces match of entire expression + generators[re] = std::make_shared(re); + } + + return pre_arg; + } + +private: + Poco::Logger * logger = nullptr; + + Regexp * root = nullptr; + Generators generators; +}; + +} + + +namespace DB +{ + +void RandomStringGeneratorByRegexp::RegexpPtrDeleter::operator() (re2::Regexp * re) const noexcept +{ + re->Decref(); +} + +RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(String re_str, bool logging) +{ + re2::RE2::Options options; + options.set_case_sensitive(true); + auto flags = static_cast(options.ParseFlags()); + + re2::RegexpStatus status; + regexp.reset(re2::Regexp::Parse(re_str, flags, &status)); + + if (!regexp) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Error parsing regexp '{}': {}", + re_str, status.Text()); + + regexp.reset(regexp->Simplify()); + + auto walker = re2::RandomStringPrepareWalker(logging); + walker.Walk(regexp.get(), {}); + generatorFunc = walker.getGenerator(); + + { + auto test_check = generate(); + auto matched = RE2::FullMatch(test_check, re2::RE2(re_str)); + if (!matched) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Generator is unable to produce random string for regexp '{}': {}", + re_str, test_check); + } +} + +String RandomStringGeneratorByRegexp::generate() const +{ + chassert(generatorFunc); + return generatorFunc(); +} + +} diff --git a/src/Common/MatchGenerator.h b/src/Common/MatchGenerator.h new file mode 100644 index 00000000000..1078c25774c --- /dev/null +++ b/src/Common/MatchGenerator.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + +namespace re2 +{ + class Regexp; +} + +namespace DB +{ + +class RandomStringGeneratorByRegexp +{ +public: + RandomStringGeneratorByRegexp(String re_str, bool logging); + String generate() const; + +private: + struct RegexpPtrDeleter + { + void operator()(re2::Regexp * re) const noexcept; + }; + using RegexpPtr = std::unique_ptr; + + RegexpPtr regexp; + std::function generatorFunc; +}; + +} diff --git a/src/Common/ObjectStorageKey.cpp b/src/Common/ObjectStorageKey.cpp index ca5617c8aa2..feda1d9ac29 100644 --- a/src/Common/ObjectStorageKey.cpp +++ b/src/Common/ObjectStorageKey.cpp @@ -65,4 +65,5 @@ ObjectStorageKey ObjectStorageKey::createAsAbsolute(String key_) object_key.is_relative = false; return object_key; } + } diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp new file mode 100644 index 00000000000..7d81726344a --- /dev/null +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -0,0 +1,97 @@ +#include "ObjectStorageKeyGenerator.h" + +#include +#include + +#include +#include + + +class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator +{ +public: + GeneratorWithTemplate(String key_template_) + : key_template(std::move(key_template_)) + , re_gen(key_template, /*logging*/ false) + { + } + + DB::ObjectStorageKey generate(const String &) const override + { + return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); + } + +private: + String key_template; + DB::RandomStringGeneratorByRegexp re_gen; +}; + + +class GeneratorWithPrefix : public DB::IObjectStorageKeysGenerator +{ +public: + GeneratorWithPrefix(String key_prefix_) + : key_prefix(std::move(key_prefix_)) + {} + + DB::ObjectStorageKey generate(const String &) const override + { + /// Path to store the new S3 object. + + /// Total length is 32 a-z characters for enough randomness. + /// First 3 characters are used as a prefix for + /// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/ + + constexpr size_t key_name_total_size = 32; + constexpr size_t key_name_prefix_size = 3; + + /// Path to store new S3 object. + String key = fmt::format("{}/{}", + DB::getRandomASCIIString(key_name_prefix_size), + DB::getRandomASCIIString(key_name_total_size - key_name_prefix_size)); + + /// what ever key_prefix value is, consider that key as relative + return DB::ObjectStorageKey::createAsRelative(key_prefix, key); + } + +private: + String key_prefix; +}; + + +class GeneratorAsIsWithPrefix : public DB::IObjectStorageKeysGenerator +{ +public: + GeneratorAsIsWithPrefix(String key_prefix_) + : key_prefix(std::move(key_prefix_)) + {} + + DB::ObjectStorageKey generate(const String & path) const override + { + return DB::ObjectStorageKey::createAsRelative(key_prefix, path); + } + +private: + String key_prefix; +}; + + +namespace DB +{ + +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorAsIsWithPrefix(String key_prefix) +{ + return std::make_shared(std::move(key_prefix)); +} + +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByPrefix(String key_prefix) +{ + return std::make_shared(std::move(key_prefix)); +} + +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByTemplate(String key_template) +{ + return std::make_shared(std::move(key_template)); +} + +} diff --git a/src/Common/ObjectStorageKeyGenerator.h b/src/Common/ObjectStorageKeyGenerator.h new file mode 100644 index 00000000000..29f2a4a22c2 --- /dev/null +++ b/src/Common/ObjectStorageKeyGenerator.h @@ -0,0 +1,22 @@ +#pragma once + +#include "ObjectStorageKey.h" +#include + +namespace DB +{ + +class IObjectStorageKeysGenerator +{ +public: + virtual ObjectStorageKey generate(const String & path) const = 0; + virtual ~IObjectStorageKeysGenerator() = default; +}; + +using ObjectStorageKeysGeneratorPtr = std::shared_ptr; + +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorAsIsWithPrefix(String key_prefix); +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByPrefix(String key_prefix); +ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByTemplate(String key_template); + +} diff --git a/src/Common/tests/gtest_generate_random_by_regexp.cpp b/src/Common/tests/gtest_generate_random_by_regexp.cpp new file mode 100644 index 00000000000..74c17a87a10 --- /dev/null +++ b/src/Common/tests/gtest_generate_random_by_regexp.cpp @@ -0,0 +1,97 @@ +#include +#include +#include +#include + +#include + +void routine(String s) +{ + std::cerr << "case '"<< s << "'"; + auto gen = DB::RandomStringGeneratorByRegexp(s, /*logging*/ true); + [[maybe_unused]] auto res = gen.generate(); + std::cerr << " result '"<< res << "'" << std::endl; +} + +TEST(GenerateRandomString, Positive) +{ + routine("."); + routine("[[:xdigit:]]"); + routine("[0-9a-f]"); + routine("[a-z]"); + routine("prefix-[0-9a-f]-suffix"); + routine("prefix-[a-z]-suffix"); + routine("[0-9a-f]{3}"); + routine("prefix-[0-9a-f]{3}-suffix"); + routine("prefix-[a-z]{3}-suffix/[0-9a-f]{20}"); + routine("left|right"); + routine("[a-z]{0,3}"); + routine("just constant string"); + routine("[a-z]?"); + routine("[a-z]*"); + routine("[a-z]+"); + routine("[^a-z]"); + routine("[[:lower:]]{3}/suffix"); + routine("prefix-(A|B|[0-9a-f]){3}"); + routine("mergetree/[a-z]{3}/[a-z]{29}"); +} + +TEST(GenerateRandomString, Negative) +{ + EXPECT_THROW(routine("[[:do_not_exists:]]"), DB::Exception); + EXPECT_THROW(routine("[:do_not_exis..."), DB::Exception); + EXPECT_THROW(routine("^abc"), DB::Exception); +} + +TEST(GenerateRandomString, DifferentResult) +{ + std::cerr << "100 different keys" << std::endl; + auto gen = DB::RandomStringGeneratorByRegexp("prefix-[a-z]{3}-suffix/[0-9a-f]{20}", /*logging*/ true); + std::set deduplicate; + for (int i = 0; i < 100; ++i) + ASSERT_TRUE(deduplicate.insert(gen.generate()).second); + std::cerr << "100 different keys: ok" << std::endl; +} + +TEST(GenerateRandomString, FullRange) +{ + std::cerr << "all possible letters" << std::endl; + auto gen = DB::RandomStringGeneratorByRegexp("[a-z]", /*logging*/ false); + std::set deduplicate; + int count = 'z' - 'a' + 1; + while (deduplicate.size() < count) + if (deduplicate.insert(gen.generate()).second) + std::cerr << " +1 "; + std::cerr << "all possible letters, ok" << std::endl; +} + +UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator) +{ + String path = "some_path"; + + Stopwatch watch; + + for (int i = 0; i < 10000; ++i) + { + [[ maybe_unused ]] auto result = generator->generate(path).serialize(); + } + + return watch.elapsedMilliseconds(); +} + +TEST(ObjectStorageKey, Performance) +{ + auto elapsed_old = elapsed(DB::createObjectStorageKeysGeneratorByPrefix("mergetree/")); + std::cerr << "old: " << elapsed_old << std::endl; + + auto elapsed_new = elapsed(DB::createObjectStorageKeysGeneratorByTemplate("mergetree/[a-z]{3}/[a-z]{29}")); + std::cerr << "new: " << elapsed_new << std::endl; + + if (elapsed_new > elapsed_old) + { + auto diff = elapsed_new - elapsed_old; + std::cerr << "slow ratio: " << float(diff) / elapsed_old << std::endl; + ASSERT_GT(1.1 * elapsed_old, elapsed_new); + } + +} diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index 3271a190193..881f7a46c16 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) { readIntText(version, buf); + assertChar('\n', buf); if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_FULL_OBJECT_KEY) throw Exception( @@ -27,8 +28,6 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) "Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}", metadata_file_path, toString(version), toString(VERSION_FULL_OBJECT_KEY)); - assertChar('\n', buf); - UInt32 keys_count; readIntText(keys_count, buf); assertChar('\t', buf); @@ -122,6 +121,7 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const chassert(write_version >= VERSION_ABSOLUTE_PATHS && write_version <= VERSION_FULL_OBJECT_KEY); writeIntText(write_version, buf); + writeChar('\n', buf); writeIntText(keys_with_meta.size(), buf); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index beb8a400632..6a091471888 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -19,7 +19,6 @@ #include -#include #include #include #include @@ -556,27 +555,12 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( return std::make_unique( std::move(new_client), std::move(new_s3_settings), version_id, s3_capabilities, new_namespace, - endpoint, object_key_prefix, disk_name); + endpoint, key_generator, disk_name); } -ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const +ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const { - /// Path to store the new S3 object. - - /// Total length is 32 a-z characters for enough randomness. - /// First 3 characters are used as a prefix for - /// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/ - - constexpr size_t key_name_total_size = 32; - constexpr size_t key_name_prefix_size = 3; - - /// Path to store new S3 object. - String key = fmt::format("{}/{}", - getRandomASCIIString(key_name_prefix_size), - getRandomASCIIString(key_name_total_size - key_name_prefix_size)); - - /// what ever key_prefix value is, consider that key as relative - return ObjectStorageKey::createAsRelative(object_key_prefix, key); + return key_generator->generate(path); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index c8b3aeaca28..caa4beaba3b 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -39,7 +40,6 @@ struct S3ObjectStorageSettings bool read_only; }; - class S3ObjectStorage : public IObjectStorage { private: @@ -53,10 +53,10 @@ private: const S3Capabilities & s3_capabilities_, String bucket_, String connection_string, - String object_key_prefix_, + ObjectStorageKeysGeneratorPtr key_generator_, const String & disk_name_) : bucket(std::move(bucket_)) - , object_key_prefix(std::move(object_key_prefix_)) + , key_generator(std::move(key_generator_)) , disk_name(disk_name_) , client(std::move(client_)) , s3_settings(std::move(s3_settings_)) @@ -179,7 +179,7 @@ private: private: std::string bucket; - String object_key_prefix; + ObjectStorageKeysGeneratorPtr key_generator; std::string disk_name; MultiVersion client; @@ -199,11 +199,6 @@ private: class S3PlainObjectStorage : public S3ObjectStorage { public: - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override - { - return ObjectStorageKey::createAsRelative(object_key_prefix, path); - } - std::string getName() const override { return "S3PlainObjectStorage"; } template diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 7543fb94331..a35a1eb2a82 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -91,6 +91,60 @@ private: } }; +std::pair getPrefixAndKeyGenerator( + String type, const S3::URI & uri, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +{ + if (type == "s3_plain") + return {uri.key, createObjectStorageKeysGeneratorAsIsWithPrefix(uri.key)}; + + chassert(type == "s3"); + + bool storage_metadata_write_full_object_key = DiskObjectStorageMetadata::getWriteFullObjectKeySetting(); + bool send_metadata = config.getBool(config_prefix + ".send_metadata", false); + + if (send_metadata && storage_metadata_write_full_object_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Wrong configuration in {}. " + "s3 does not supports feature 'send_metadata' with feature 'storage_metadata_write_full_object_key'.", + config_prefix); + + String object_key_compatibility_prefix = config.getString(config_prefix + ".key_compatibility_prefix", String()); + String object_key_template = config.getString(config_prefix + ".key_template", String()); + + if (object_key_template.empty()) + { + if (!object_key_compatibility_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Wrong configuration in {}. " + "Setting 'key_compatibility_prefix' can be defined only with setting 'key_template'.", + config_prefix); + + return {uri.key, createObjectStorageKeysGeneratorByPrefix(uri.key)}; + } + + if (send_metadata) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Wrong configuration in {}. " + "s3 does not supports send_metadata with setting 'key_template'.", + config_prefix); + + if (!storage_metadata_write_full_object_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Wrong configuration in {}. " + "Feature 'storage_metadata_write_full_object_key' has to be enabled in order to use setting 'key_template'.", + config_prefix); + + if (!uri.key.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Wrong configuration in {}. " + "URI.key is forbidden with settings 'key_template', use setting 'key_compatibility_prefix' instead'. " + "URI.key: '{}', bucket: '{}'. ", + config_prefix, + uri.key, uri.bucket); + + return {object_key_compatibility_prefix, createObjectStorageKeysGeneratorByTemplate(object_key_template)}; +} + } void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) @@ -104,7 +158,8 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) { String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); - if (!uri.key.ends_with('/')) + // an empty key remains empty + if (!uri.key.empty() && !uri.key.ends_with('/')) uri.key.push_back('/'); S3Capabilities s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); @@ -113,6 +168,8 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) String type = config.getString(config_prefix + ".type"); chassert(type == "s3" || type == "s3_plain"); + auto [object_key_compatibility_prefix, object_key_generator] = getPrefixAndKeyGenerator(type, uri, config, config_prefix); + MetadataStoragePtr metadata_storage; auto settings = getSettings(config, config_prefix, context); auto client = getClient(config, config_prefix, context, *settings); @@ -128,20 +185,18 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain does not supports send_metadata"); s3_storage = std::make_shared( - std::move(client), std::move(settings), - uri.version_id, s3_capabilities, - uri.bucket, uri.endpoint, uri.key, name); - metadata_storage = std::make_shared(s3_storage, uri.key); + std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint, object_key_generator, name); + + metadata_storage = std::make_shared(s3_storage, object_key_compatibility_prefix); } else { s3_storage = std::make_shared( - std::move(client), std::move(settings), - uri.version_id, s3_capabilities, - uri.bucket, uri.endpoint, uri.key, name); + std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint, object_key_generator, name); auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); - metadata_storage = std::make_shared(metadata_disk, uri.key); + + metadata_storage = std::make_shared(metadata_disk, object_key_compatibility_prefix); } /// NOTE: should we still perform this check for clickhouse-disks? @@ -164,7 +219,7 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) DiskObjectStoragePtr s3disk = std::make_shared( name, - uri.key, + uri.key, /// might be empty type == "s3" ? "DiskS3" : "DiskS3Plain", std::move(metadata_storage), std::move(s3_storage), diff --git a/tests/integration/test_remote_blobs_naming/configs/setting.xml b/tests/integration/test_remote_blobs_naming/configs/setting.xml new file mode 100644 index 00000000000..408fa36fdd3 --- /dev/null +++ b/tests/integration/test_remote_blobs_naming/configs/setting.xml @@ -0,0 +1,11 @@ + + + + + + 1 + 1 + + + + diff --git a/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml b/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml index 31c6a3bf968..cade16dc9c6 100644 --- a/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml +++ b/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml @@ -9,7 +9,7 @@ s3 - http://minio1:9001/root/data/ + http://minio1:9001/root/old-style-prefix/with-several-section/ minio minio123 @@ -30,7 +30,6 @@ -

diff --git a/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml b/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml new file mode 100644 index 00000000000..dc1bab458f0 --- /dev/null +++ b/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml @@ -0,0 +1,61 @@ + + + + + test + + + + + + s3 + http://minio1:9001/root/old-style-prefix/with-several-section/ + minio + minio123 + + + s3_plain + http://minio1:9001/root/data/s3_pain_key_prefix + minio + minio123 + true + + + s3 + http://minio1:9001/root/ + minio + minio123 + old-style-prefix/with-several-section + [a-z]{3}-first-random-part/constant-part/[a-z]{3}/[a-z]{29} + + + + + + +
+ s3 +
+
+
+ + +
+ s3_plain +
+
+
+ + +
+ s3_template_key +
+
+
+
+
+ + + s3 + +
diff --git a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py index 485bf73dad1..5f91c0e72ef 100644 --- a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py +++ b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py @@ -27,7 +27,7 @@ def cluster(): "new_node", main_configs=[ "configs/new_node.xml", - "configs/storage_conf.xml", + "configs/storage_conf_new.xml", ], user_configs=[ "configs/settings.xml", @@ -49,6 +49,7 @@ def cluster(): with_zookeeper=True, stay_alive=True, ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -294,3 +295,33 @@ def test_log_table(cluster, storage_policy): assert "4" == node.query("SELECT count() FROM test_log_table").strip() node.query("DROP TABLE test_log_table SYNC") + + +def test_template_key(cluster): + old_node = cluster.instances["node"] + new_node = cluster.instances["new_node"] + + def get_create_statement(storage_policy): + create_table_statement = f""" + CREATE TABLE test_template_key ( + id Int64, + val String + ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_template_key', '{{replica}}') + PARTITION BY id + ORDER BY (id, val) + SETTINGS + storage_policy='{storage_policy}' + """ + return create_table_statement + + old_node.query(get_create_statement("s3")) + new_node.query(get_create_statement("s3_template_key")) + + old_node.query("INSERT INTO test_template_key VALUES (0, 'a')") + new_node.query("INSERT INTO test_template_key VALUES (1, 'b')") + + old_node.query("SYSTEM SYNC REPLICA test_template_key") + new_node.query("SYSTEM SYNC REPLICA test_template_key") + + assert "2" == old_node.query("SELECT count() FROM test_template_key").strip() + assert "2" == new_node.query("SELECT count() FROM test_template_key").strip() From a409b6c420517baec4b0ff6f2be09f1a20f1c204 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 20 Nov 2023 19:15:40 +0100 Subject: [PATCH 060/116] speed up generation --- src/Common/MatchGenerator.cpp | 196 +++++++++++------- .../tests/gtest_generate_random_by_regexp.cpp | 16 +- 2 files changed, 128 insertions(+), 84 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 5fa644c20f9..21762c65cff 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -49,7 +49,7 @@ class RandomStringPrepareWalker : public Regexp::Walker private: static constexpr int ImplicitMax = 100; - using Children = std::vector; + using Children = std::vector; class Generators; @@ -57,187 +57,226 @@ private: class NodeFunction { public: - virtual String operator() (const Generators& generators) = 0; + virtual String operator() () = 0; virtual ~NodeFunction() = default; }; using NodeFunctionPtr = std::shared_ptr; + using NodeFuncs = std::vector; + + static NodeFuncs getFuncs(Children children_, const Generators & generators_) + { + NodeFuncs result; + result.reserve(children_.size()); + + for (auto * child: children_) + { + result.push_back(generators_.at(child)); + } + + return result; + } class Generators: public std::map {}; class RegexpConcatFunction : public NodeFunction { public: - RegexpConcatFunction(Children children_) - : children(std::move(children_)) - {} - - String operator () (const Generators& generators_) override + RegexpConcatFunction(Children children_, const Generators & generators_) + : children(getFuncs(children_, generators_)) { - String result; - for (auto child: children) + } + + String operator () () override + { + size_t total_size = 0; + + std::vector part_result; + part_result.reserve(children.size()); + for (auto & child: children) { - auto res = generators_.at(child)->operator()(generators_); - result.append(res); + part_result.push_back(child->operator()()); + total_size += part_result.back().size(); } + + String result; + result.reserve(total_size); + for (auto & part: part_result) + { + result += part; + } + return result; } private: - Children children; + NodeFuncs children; }; class RegexpAlternateFunction : public NodeFunction { public: - RegexpAlternateFunction(Children children_) - : children(std::move(children_)) - {} + RegexpAlternateFunction(Children children_, const Generators & generators_) + : children(getFuncs(children_, generators_)) + { + } - String operator () (const Generators& generators_) override + String operator () () override { std::uniform_int_distribution distribution(0, static_cast(children.size()-1)); int chosen = distribution(thread_local_rng); - return generators_.at(children[chosen])->operator()(generators_); + return children[chosen]->operator()(); } private: - Children children; + NodeFuncs children; }; class RegexpRepeatFunction : public NodeFunction { public: - RegexpRepeatFunction(Regexp * re_, int min_repeat_, int max_repeat_) - : re(re_) + RegexpRepeatFunction(Regexp * re_, const Generators & generators_, int min_repeat_, int max_repeat_) + : func(generators_.at(re_)) , min_repeat(min_repeat_) , max_repeat(max_repeat_) - {} + { + } - String operator () (const Generators& generators_) override + String operator () () override { std::uniform_int_distribution distribution(min_repeat, max_repeat); int chosen = distribution(thread_local_rng); String result; for (int i = 0; i < chosen; ++i) - result.append(generators_.at(re)->operator()(generators_)); + result += func->operator()(); return result; } private: - Regexp * re; + NodeFunctionPtr func; int min_repeat = 0; int max_repeat = 0; }; class RegexpCharClassFunction : public NodeFunction { + using CharRanges = std::vector>; + public: RegexpCharClassFunction(Regexp * re_) - : re(re_) - {} - - String operator () (const Generators&) override { - CharClass * cc = re->cc(); + CharClass * cc = re_->cc(); + chassert(cc); if (cc->empty()) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "kRegexpCharClass is empty"); - std::uniform_int_distribution distribution(1, cc->size()); + char_count = cc->size(); + char_ranges.reserve(std::distance(cc->begin(), cc->end())); + + for (auto it = cc->begin(); it != cc->end(); ++it) + { + char_ranges.emplace_back(it->lo, it->hi); + } + } + + String operator () () override + { + std::uniform_int_distribution distribution(1, char_count); int chosen = distribution(thread_local_rng); int count_down = chosen; - auto it = cc->begin(); - for (; it != cc->end(); ++it) + auto it = char_ranges.begin(); + for (; it != char_ranges.end(); ++it) { - auto range_len = it->hi - it->lo + 1; + auto [lo, hi] = *it; + auto range_len = hi - lo + 1; if (count_down <= range_len) break; count_down -= range_len; } - if (it == cc->end()) + if (it == char_ranges.end()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, - "Unable to choose the rune. Runes {}, chosen {}", - cc->size(), chosen); + "Unable to choose the rune. Runes {}, ranges {}, chosen {}", + char_count, char_ranges.size(), chosen); - Rune r = it->lo + count_down - 1; - char buffer[UTFmax+1]; - buffer[re2::runetochar(buffer, &r)] = 0; - return String(buffer); + auto [lo, _] = *it; + Rune r = lo + count_down - 1; + int n = re2::runetochar(buffer, &r); + return String(buffer, n); } private: - Regexp * re; + char buffer[UTFmax]; + int char_count = 0; + CharRanges char_ranges; }; class RegexpLiteralStringFunction : public NodeFunction { public: RegexpLiteralStringFunction(Regexp * re_) - : re(re_) - {} - - String operator () (const Generators&) override { - if (re->nrunes() == 0) - return String(); + if (re_->nrunes() == 0) + return; - String result; - char buffer[UTFmax+1]; - for (int i = 0; i < re->nrunes(); ++i) + char buffer[UTFmax]; + for (int i = 0; i < re_->nrunes(); ++i) { - buffer[re2::runetochar(buffer, &re->runes()[i])] = 0; - result.append(buffer); + int n = re2::runetochar(buffer, &re_->runes()[i]); + literal_string += String(buffer, n); } - return result; + } + + String operator () () override + { + return literal_string; } private: - Regexp * re; + String literal_string; }; class RegexpLiteralFunction : public NodeFunction { public: RegexpLiteralFunction(Regexp * re_) - : re(re_) - {} - - String operator () (const Generators&) override { - String result; - char buffer[UTFmax+1]; + char buffer[UTFmax]; - Rune r = re->rune(); - buffer[re2::runetochar(buffer, &r)] = 0; - result.append(buffer); + Rune r = re_->rune(); + int n = re2::runetochar(buffer, &r); + literal = String(buffer, n); + } - return result; + String operator () () override + { + return literal; } private: - Regexp * re; + String literal; }; class ThrowExceptionFunction : public NodeFunction { public: ThrowExceptionFunction(Regexp * re_) - : re(re_) - {} + : operation(magic_enum::enum_name(re_->op())) + { + } - String operator () (const Generators&) override + String operator () () override { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, "RandomStringPrepareWalker: regexp node '{}' is not supported for generating a random match", - magic_enum::enum_name(re->op())); + operation); } private: - Regexp * re; + String operation; }; @@ -257,8 +296,8 @@ public: if (generators.size() == 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no generators"); - auto result = [generators_ = std::move(generators), root_ = std::move(root)] () -> String { - return generators_.at(root_)->operator()(generators_); + auto result = [root_func = generators.at(root)] () -> String { + return root_func->operator()(); }; root = nullptr; @@ -309,22 +348,22 @@ private: switch (re->op()) { case kRegexpConcat: // Matches concatenation of sub_[0..nsub-1]. - generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args)); + generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args), generators); break; case kRegexpAlternate: // Matches union of sub_[0..nsub-1]. - generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args)); + generators[re] = std::make_shared(CopyChildrenArgs(child_args, nchild_args), generators); break; case kRegexpQuest: // Matches sub_[0] zero or one times. chassert(nchild_args == 1); - generators[re] = std::make_shared(child_args[0], 0, 1); + generators[re] = std::make_shared(child_args[0], generators, 0, 1); break; case kRegexpStar: // Matches sub_[0] zero or more times. chassert(nchild_args == 1); - generators[re] = std::make_shared(child_args[0], 0, ImplicitMax); + generators[re] = std::make_shared(child_args[0], generators, 0, ImplicitMax); break; case kRegexpPlus: // Matches sub_[0] one or more times. chassert(nchild_args == 1); - generators[re] = std::make_shared(child_args[0], 1, ImplicitMax); + generators[re] = std::make_shared(child_args[0], generators, 1, ImplicitMax); break; case kRegexpCharClass: // Matches character class given by cc_. chassert(nchild_args == 0); @@ -340,7 +379,7 @@ private: break; case kRegexpCapture: // Parenthesized (capturing) subexpression. chassert(nchild_args == 1); - generators[re] = generators[child_args[0]]; + generators[re] = generators.at(child_args[0]); break; case kRegexpNoMatch: // Matches no strings. @@ -383,6 +422,7 @@ RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(String re_str, bool { re2::RE2::Options options; options.set_case_sensitive(true); + options.set_encoding(re2::RE2::Options::EncodingLatin1); auto flags = static_cast(options.ParseFlags()); re2::RegexpStatus status; diff --git a/src/Common/tests/gtest_generate_random_by_regexp.cpp b/src/Common/tests/gtest_generate_random_by_regexp.cpp index 74c17a87a10..70e806b023c 100644 --- a/src/Common/tests/gtest_generate_random_by_regexp.cpp +++ b/src/Common/tests/gtest_generate_random_by_regexp.cpp @@ -71,7 +71,7 @@ UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator) Stopwatch watch; - for (int i = 0; i < 10000; ++i) + for (int i = 0; i < 20000; ++i) { [[ maybe_unused ]] auto result = generator->generate(path).serialize(); } @@ -81,17 +81,21 @@ UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator) TEST(ObjectStorageKey, Performance) { - auto elapsed_old = elapsed(DB::createObjectStorageKeysGeneratorByPrefix("mergetree/")); + auto elapsed_old = elapsed(DB::createObjectStorageKeysGeneratorByPrefix( + "xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/")); std::cerr << "old: " << elapsed_old << std::endl; - auto elapsed_new = elapsed(DB::createObjectStorageKeysGeneratorByTemplate("mergetree/[a-z]{3}/[a-z]{29}")); + auto elapsed_new = elapsed(DB::createObjectStorageKeysGeneratorByTemplate( + "xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/[a-z]{3}/[a-z]{29}")); std::cerr << "new: " << elapsed_new << std::endl; if (elapsed_new > elapsed_old) { - auto diff = elapsed_new - elapsed_old; - std::cerr << "slow ratio: " << float(diff) / elapsed_old << std::endl; - ASSERT_GT(1.1 * elapsed_old, elapsed_new); + if (elapsed_new > elapsed_old) + std::cerr << "slow ratio: +" << float(elapsed_new) / elapsed_old << std::endl; + else + std::cerr << "fast ratio: " << float(elapsed_old) / elapsed_new << std::endl; + ASSERT_LT(elapsed_new, 2 * elapsed_old); } } From f7d033cac2318c20bd51ed3b77a350c6a3bbe73a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 20 Nov 2023 21:20:14 +0100 Subject: [PATCH 061/116] speed up generation, less allocations --- src/Common/MatchGenerator.cpp | 126 +++++++++++++----- .../tests/gtest_generate_random_by_regexp.cpp | 6 +- 2 files changed, 95 insertions(+), 37 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 21762c65cff..12afa0fd6e7 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -57,7 +57,8 @@ private: class NodeFunction { public: - virtual String operator() () = 0; + virtual size_t operator() (char * out, size_t size) = 0; + virtual size_t getRequiredSize() = 0; virtual ~NodeFunction() = default; }; @@ -87,26 +88,28 @@ private: { } - String operator () () override + size_t operator () (char * out, size_t size) override { size_t total_size = 0; - std::vector part_result; - part_result.reserve(children.size()); for (auto & child: children) { - part_result.push_back(child->operator()()); - total_size += part_result.back().size(); + size_t consumed = child->operator()(out, size); + chassert(consumed <= size); + out += consumed; + size -= consumed; + total_size += consumed; } - String result; - result.reserve(total_size); - for (auto & part: part_result) - { - result += part; - } + return total_size; + } - return result; + size_t getRequiredSize() override + { + size_t total_size = 0; + for (auto & child: children) + total_size += child->getRequiredSize(); + return total_size; } private: @@ -121,11 +124,21 @@ private: { } - String operator () () override + size_t operator () (char * out, size_t size) override { std::uniform_int_distribution distribution(0, static_cast(children.size()-1)); int chosen = distribution(thread_local_rng); - return children[chosen]->operator()(); + size_t consumed = children[chosen]->operator()(out, size); + chassert(consumed <= size); + return consumed; + } + + size_t getRequiredSize() override + { + size_t total_size = 0; + for (auto & child: children) + total_size = std::max(total_size, child->getRequiredSize()); + return total_size; } private: @@ -142,15 +155,26 @@ private: { } - String operator () () override + size_t operator () (char * out, size_t size) override { std::uniform_int_distribution distribution(min_repeat, max_repeat); - int chosen = distribution(thread_local_rng); + int ntimes = distribution(thread_local_rng); - String result; - for (int i = 0; i < chosen; ++i) - result += func->operator()(); - return result; + size_t total_size = 0; + for (int i = 0; i < ntimes; ++i) + { + size_t consumed =func->operator()(out, size); + chassert(consumed <= size); + out += consumed; + size -= consumed; + total_size += consumed; + } + return total_size; + } + + size_t getRequiredSize() override + { + return max_repeat * func->getRequiredSize(); } private: @@ -180,8 +204,10 @@ private: } } - String operator () () override + size_t operator () (char * out, size_t size) override { + chassert(UTFmax <= size); + std::uniform_int_distribution distribution(1, char_count); int chosen = distribution(thread_local_rng); int count_down = chosen; @@ -203,12 +229,16 @@ private: auto [lo, _] = *it; Rune r = lo + count_down - 1; - int n = re2::runetochar(buffer, &r); - return String(buffer, n); + int n = re2::runetochar(out, &r); + return n; + } + + size_t getRequiredSize() override + { + return UTFmax; } private: - char buffer[UTFmax]; int char_count = 0; CharRanges char_ranges; }; @@ -229,9 +259,17 @@ private: } } - String operator () () override + size_t operator () (char * out, size_t size) override { - return literal_string; + chassert(literal_string.size() <= size); + + memcpy(out, literal_string.data(), literal_string.size()); + return literal_string.size(); + } + + size_t getRequiredSize() override + { + return literal_string.size(); } private: @@ -250,9 +288,17 @@ private: literal = String(buffer, n); } - String operator () () override + size_t operator () (char * out, size_t size) override { - return literal; + chassert(literal.size() <= size); + + memcpy(out, literal.data(), literal.size()); + return literal.size(); + } + + size_t getRequiredSize() override + { + return literal.size(); } private: @@ -267,7 +313,7 @@ private: { } - String operator () () override + size_t operator () (char *, size_t) override { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, @@ -275,6 +321,11 @@ private: operation); } + size_t getRequiredSize() override + { + return 0; + } + private: String operation; }; @@ -296,14 +347,21 @@ public: if (generators.size() == 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no generators"); - auto result = [root_func = generators.at(root)] () -> String { - return root_func->operator()(); + auto root_func = generators.at(root); + auto required_buffer_size = root_func->getRequiredSize(); + auto generator_func = [=] () + -> String + { + auto buffer = String(required_buffer_size, '\0'); + size_t size = root_func->operator()(buffer.data(), buffer.size()); + buffer.resize(size); + return buffer; }; root = nullptr; - generators.clear(); + generators = {}; - return std::move(result); + return std::move(generator_func); } private: diff --git a/src/Common/tests/gtest_generate_random_by_regexp.cpp b/src/Common/tests/gtest_generate_random_by_regexp.cpp index 70e806b023c..67ccf1ffe84 100644 --- a/src/Common/tests/gtest_generate_random_by_regexp.cpp +++ b/src/Common/tests/gtest_generate_random_by_regexp.cpp @@ -71,12 +71,12 @@ UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator) Stopwatch watch; - for (int i = 0; i < 20000; ++i) + for (int i = 0; i < 100000; ++i) { [[ maybe_unused ]] auto result = generator->generate(path).serialize(); } - return watch.elapsedMilliseconds(); + return watch.elapsedMicroseconds(); } TEST(ObjectStorageKey, Performance) @@ -95,7 +95,7 @@ TEST(ObjectStorageKey, Performance) std::cerr << "slow ratio: +" << float(elapsed_new) / elapsed_old << std::endl; else std::cerr << "fast ratio: " << float(elapsed_old) / elapsed_new << std::endl; - ASSERT_LT(elapsed_new, 2 * elapsed_old); + ASSERT_LT(elapsed_new, 1.2 * elapsed_old); } } From 680ce20509b0493791223abb8ca73eacbffa9f00 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 20 Dec 2023 16:24:24 +0100 Subject: [PATCH 062/116] build fix --- src/Common/ObjectStorageKeyGenerator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index 7d81726344a..ffac185ae3f 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -10,7 +10,7 @@ class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator { public: - GeneratorWithTemplate(String key_template_) + explicit GeneratorWithTemplate(String key_template_) : key_template(std::move(key_template_)) , re_gen(key_template, /*logging*/ false) { @@ -62,7 +62,7 @@ private: class GeneratorAsIsWithPrefix : public DB::IObjectStorageKeysGenerator { public: - GeneratorAsIsWithPrefix(String key_prefix_) + explicit GeneratorAsIsWithPrefix(String key_prefix_) : key_prefix(std::move(key_prefix_)) {} From 8ab4aa3423e83104280a9516718721fc1f1cfc54 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 2 Jan 2024 19:59:59 +0100 Subject: [PATCH 063/116] enable randomization setting --- docker/test/stateless/stress_tests.lib | 3 + docker/test/stress/run.sh | 1 + .../ObjectStorages/DiskObjectStorage.cpp | 6 - ...torage_policy_with_template_object_key.xml | 32 +++ .../storage_metadata_with_full_object_key.xml | 5 + tests/config/install.sh | 13 +- .../configs/storage_conf.xml | 19 +- .../configs/storage_conf_new.xml | 14 +- .../test_backward_compatibility.py | 197 +++++++++++++----- 9 files changed, 215 insertions(+), 75 deletions(-) create mode 100644 tests/config/config.d/s3_storage_policy_with_template_object_key.xml create mode 100644 tests/config/config.d/storage_metadata_with_full_object_key.xml diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 6f0dabb5207..e372d4db465 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -9,6 +9,9 @@ FAIL="\tFAIL\t\\N\t" FAILURE_CONTEXT_LINES=100 FAILURE_CONTEXT_MAX_LINE_WIDTH=300 +export S3_OBJECT_KEY_TYPES=("generate-suffix" "generate-full-key" "generate-template-key") +export USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE="${S3_OBJECT_KEY_TYPES[0]}" + function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 67056cc1bc1..beacd181e1d 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -194,6 +194,7 @@ stop # Let's enable S3 storage by default export USE_S3_STORAGE_FOR_MERGE_TREE=1 export ZOOKEEPER_FAULT_INJECTION=1 +export USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE="${S3_OBJECT_KEY_TYPES[$((RANDOM % ${#S3_OBJECT_KEY_TYPES[@]}))]}" configure # But we still need default disk because some tables loaded only into it diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index c3baf3fdbda..6962248c7e1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -258,12 +258,6 @@ String DiskObjectStorage::getUniqueId(const String & path) const bool DiskObjectStorage::checkUniqueId(const String & id) const { - if (!id.starts_with(object_key_prefix)) - { - LOG_DEBUG(log, "Blob with id {} doesn't start with blob storage prefix {}, Stack {}", id, object_key_prefix, StackTrace().toString()); - return false; - } - auto object = StoredObject(id); return object_storage->exists(object); } diff --git a/tests/config/config.d/s3_storage_policy_with_template_object_key.xml b/tests/config/config.d/s3_storage_policy_with_template_object_key.xml new file mode 100644 index 00000000000..834f5102da1 --- /dev/null +++ b/tests/config/config.d/s3_storage_policy_with_template_object_key.xml @@ -0,0 +1,32 @@ + + + + + s3 + http://localhost:11111/test/ + clickhouse + clickhouse + test + + [a-z]{3}-first-random-part/new-style-prefix/[a-z]{3}/[a-z]{29} + + + cache + 1Gi + cached_s3/ + s3 + + + + + +
cached_s3
+
+
+
+
+ + s3 + + cached_s3 +
diff --git a/tests/config/config.d/storage_metadata_with_full_object_key.xml b/tests/config/config.d/storage_metadata_with_full_object_key.xml new file mode 100644 index 00000000000..2bb8d49ec4b --- /dev/null +++ b/tests/config/config.d/storage_metadata_with_full_object_key.xml @@ -0,0 +1,5 @@ + + + + 1 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 2f9fd44c9b0..8cbed155a72 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -141,7 +141,18 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th fi if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then - ln -sf $SRC_PATH/config.d/s3_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ + case "$USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE" in + "generate-full-key") + ln -sf $SRC_PATH/config.d/storage_metadata_with_full_object_key.xml $DEST_SERVER_PATH/config.d/ + ;; + "generate-template-key") + ln -sf $SRC_PATH/config.d/storage_metadata_with_full_object_key.xml $DEST_SERVER_PATH/config.d/ + ln -sf $SRC_PATH/config.d/s3_storage_policy_with_template_object_key.xml $DEST_SERVER_PATH/config.d/ + ;; + "generate-suffix"|*) + ln -sf $SRC_PATH/config.d/s3_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ + ;; + esac fi ARM="aarch64" diff --git a/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml b/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml index cade16dc9c6..e901f0df51d 100644 --- a/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml +++ b/tests/integration/test_remote_blobs_naming/configs/storage_conf.xml @@ -14,12 +14,12 @@ minio123 - s3_plain - http://minio1:9001/root/data/s3_pain_key_prefix - minio - minio123 - true - + s3_plain + http://minio1:9001/root/data/s3_pain_key_prefix + minio + minio123 + true + @@ -37,6 +37,13 @@
+ + +
+ s3 +
+
+
diff --git a/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml b/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml index dc1bab458f0..c3b515e8777 100644 --- a/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml +++ b/tests/integration/test_remote_blobs_naming/configs/storage_conf_new.xml @@ -14,19 +14,19 @@ minio123 - s3_plain - http://minio1:9001/root/data/s3_pain_key_prefix - minio - minio123 - true - + s3_plain + http://minio1:9001/root/data/s3_pain_key_prefix + minio + minio123 + true + s3 http://minio1:9001/root/ minio minio123 old-style-prefix/with-several-section - [a-z]{3}-first-random-part/constant-part/[a-z]{3}/[a-z]{29} + [a-z]{3}-first-random-part/new-style-prefix/constant-part/[a-z]{3}/[a-z]{29} diff --git a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py index 5f91c0e72ef..ae87b19c071 100644 --- a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py +++ b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py @@ -1,6 +1,8 @@ #!/usr/bin/env python3 - +from contextlib import contextmanager +from difflib import unified_diff import logging +import re import pytest import os @@ -201,8 +203,28 @@ def test_write_new_format(cluster): assert remote == object_key -@pytest.mark.parametrize("storage_policy", ["s3", "s3_plain"]) -def test_replicated_merge_tree(cluster, storage_policy): +@contextmanager +def drop_table_scope(nodes, tables, create_statements): + try: + for node in nodes: + for statement in create_statements: + node.query(statement) + yield + finally: + for node in nodes: + for table in tables: + node.query(f"DROP TABLE IF EXISTS {table} SYNC") + + +@pytest.mark.parametrize("test_case", [("s3_plain", False), + ("s3", False), + ("s3", True), + ("s3_template_key", False), + ("s3_template_key", True), + ]) +def test_replicated_merge_tree(cluster, test_case): + storage_policy, zero_copy = test_case + if storage_policy == "s3_plain": # MergeTree table doesn't work on s3_plain. Rename operation is not implemented return @@ -210,35 +232,130 @@ def test_replicated_merge_tree(cluster, storage_policy): node_old = cluster.instances["node"] node_new = cluster.instances["new_node"] + zk_table_path = f"/clickhouse/tables/test_replicated_merge_tree_{storage_policy}{'_zero_copy' if zero_copy else ''}" create_table_statement = f""" - CREATE TABLE test_replicated_merge_tree ( - id Int64, - val String - ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_replicated_merge_tree_{storage_policy}', '{{replica}}') - PARTITION BY id - ORDER BY (id, val) - SETTINGS - storage_policy='{storage_policy}' - """ + CREATE TABLE test_replicated_merge_tree ( + id Int64, + val String + ) ENGINE=ReplicatedMergeTree('{zk_table_path}', '{{replica}}') + PARTITION BY id + ORDER BY (id, val) + SETTINGS + storage_policy='{storage_policy}', + allow_remote_fs_zero_copy_replication='{1 if zero_copy else 0}' + """ - node_old.query(create_table_statement) - node_new.query(create_table_statement) + with drop_table_scope([node_old, node_new], ["test_replicated_merge_tree"], [create_table_statement]): + node_old.query("INSERT INTO test_replicated_merge_tree VALUES (0, 'a')") + node_new.query("INSERT INTO test_replicated_merge_tree VALUES (1, 'b')") - node_old.query("INSERT INTO test_replicated_merge_tree VALUES (0, 'a')") - node_new.query("INSERT INTO test_replicated_merge_tree VALUES (1, 'b')") + # node_old have to fetch metadata from node_new and vice versa + node_old.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") + node_new.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") - # node_old have to fetch metadata from node_new and vice versa - node_old.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") - node_new.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") + count_old = node_old.query("SELECT count() FROM test_replicated_merge_tree").strip() + count_new = node_new.query("SELECT count() FROM test_replicated_merge_tree").strip() - count_old = node_old.query("SELECT count() FROM test_replicated_merge_tree").strip() - count_new = node_new.query("SELECT count() FROM test_replicated_merge_tree").strip() + assert count_old == "2" + assert count_new == "2" - assert count_old == "2" - assert count_new == "2" + if not zero_copy: + return + def get_remote_pathes(node, table_name, only_remote_path=True): + uuid = node.query(f""" + SELECT uuid + FROM system.tables + WHERE name = '{table_name}' + """).strip() + assert uuid + return node.query(f""" + SELECT {"remote_path" if only_remote_path else "*"} + FROM system.remote_data_paths + WHERE + local_path LIKE '%{uuid}%' + AND local_path NOT LIKE '%format_version.txt%' + ORDER BY ALL + """).strip() - node_old.query("DROP TABLE test_replicated_merge_tree SYNC") - node_new.query("DROP TABLE test_replicated_merge_tree SYNC") + remote_pathes_old = get_remote_pathes(node_old, 'test_replicated_merge_tree') + remote_pathes_new = get_remote_pathes(node_new, 'test_replicated_merge_tree') + + assert len(remote_pathes_old) > 0 + assert remote_pathes_old == remote_pathes_new, ( + str(unified_diff(remote_pathes_old, remote_pathes_new)) + + "\n\nold:\n" + get_remote_pathes(node_old, 'test_replicated_merge_tree', False) + + "\n\nnew:\n" + get_remote_pathes(node_new, 'test_replicated_merge_tree', False) + ) + + def count_lines_with(lines, pattern): + return sum([1 for x in lines if pattern in x]) + + remore_pathes_with_old_format = count_lines_with(remote_pathes_old.split(), "old-style-prefix") + remore_pathes_with_new_format = count_lines_with(remote_pathes_old.split(), "new-style-prefix") + + if storage_policy == "s3_template_key": + assert remore_pathes_with_old_format == remore_pathes_with_new_format + assert remore_pathes_with_old_format == len(remote_pathes_old.split()) / 2 + else: + assert remore_pathes_with_old_format == len(remote_pathes_old.split()) + assert remore_pathes_with_new_format == 0 + + parts = node_old.query(""" + SELECT name + FROM system.parts + WHERE + table = 'test_replicated_merge_tree' + AND active + ORDER BY ALL + """).strip().split() + table_shared_uuid = node_old.query(f"SELECT value FROM system.zookeeper WHERE path='{zk_table_path}' and name='table_shared_id'").strip() + + part_blobs = {} + blobs_replicas = {} + + for part in parts: + blobs = node_old.query(f""" + SELECT name + FROM system.zookeeper + WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}' + ORDER BY ALL + """).strip().split() + + for blob in blobs: + replicas = node_old.query(f""" + SELECT name + FROM system.zookeeper + WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}/{blob}' + ORDER BY ALL + """).strip().split() + assert blob not in blobs_replicas + blobs_replicas[blob] = replicas + + assert part not in part_blobs + part_blobs[part] = blobs + + assert len(parts) == 2, "parts: " + str(parts) + assert len(part_blobs.keys()) == len(parts), ( + "part_blobs: " + str(part_blobs) + "; parts: " + str(parts) + ) + assert len(blobs_replicas.keys()) == len(parts), ( + "blobs_replicas: " + str(blobs_replicas) + "; parts: " + str(parts) + ) + + for replicas in blobs_replicas.values(): + assert len(replicas) == 2, "blobs_replicas: " + str(blobs_replicas) + + + for blob in blobs_replicas.keys(): + assert re.match("(old-style-prefix_with-several-section|[a-z]{3}-first-random-part_new-style-prefix_constant-part)_[a-z]{3}_[a-z]{29}", blob), ( + "blobs_replicas: " + str(blobs_replicas) + ) + + old_style_count = sum([1 for x in blobs_replicas.keys() if "old-style-prefix" in x]) + new_style_count = sum([1 for x in blobs_replicas.keys() if "new-style-prefix" in x]) + + assert (new_style_count > 0 and old_style_count == new_style_count) \ + or (new_style_count == 0 and old_style_count == len(blobs_replicas)) def switch_config_write_full_object_key(node, enable): @@ -295,33 +412,3 @@ def test_log_table(cluster, storage_policy): assert "4" == node.query("SELECT count() FROM test_log_table").strip() node.query("DROP TABLE test_log_table SYNC") - - -def test_template_key(cluster): - old_node = cluster.instances["node"] - new_node = cluster.instances["new_node"] - - def get_create_statement(storage_policy): - create_table_statement = f""" - CREATE TABLE test_template_key ( - id Int64, - val String - ) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_template_key', '{{replica}}') - PARTITION BY id - ORDER BY (id, val) - SETTINGS - storage_policy='{storage_policy}' - """ - return create_table_statement - - old_node.query(get_create_statement("s3")) - new_node.query(get_create_statement("s3_template_key")) - - old_node.query("INSERT INTO test_template_key VALUES (0, 'a')") - new_node.query("INSERT INTO test_template_key VALUES (1, 'b')") - - old_node.query("SYSTEM SYNC REPLICA test_template_key") - new_node.query("SYSTEM SYNC REPLICA test_template_key") - - assert "2" == old_node.query("SELECT count() FROM test_template_key").strip() - assert "2" == new_node.query("SELECT count() FROM test_template_key").strip() From 2861cc70ba8af65e4fbf51c19d924846edf11621 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 2 Jan 2024 20:20:44 +0100 Subject: [PATCH 064/116] fix style --- .../test_backward_compatibility.py | 156 ++++++++++++------ 1 file changed, 101 insertions(+), 55 deletions(-) diff --git a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py index ae87b19c071..8c52b05dba2 100644 --- a/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py +++ b/tests/integration/test_remote_blobs_naming/test_backward_compatibility.py @@ -216,12 +216,16 @@ def drop_table_scope(nodes, tables, create_statements): node.query(f"DROP TABLE IF EXISTS {table} SYNC") -@pytest.mark.parametrize("test_case", [("s3_plain", False), - ("s3", False), - ("s3", True), - ("s3_template_key", False), - ("s3_template_key", True), - ]) +@pytest.mark.parametrize( + "test_case", + [ + ("s3_plain", False), + ("s3", False), + ("s3", True), + ("s3_template_key", False), + ("s3_template_key", True), + ], +) def test_replicated_merge_tree(cluster, test_case): storage_policy, zero_copy = test_case @@ -245,7 +249,9 @@ def test_replicated_merge_tree(cluster, test_case): allow_remote_fs_zero_copy_replication='{1 if zero_copy else 0}' """ - with drop_table_scope([node_old, node_new], ["test_replicated_merge_tree"], [create_table_statement]): + with drop_table_scope( + [node_old, node_new], ["test_replicated_merge_tree"], [create_table_statement] + ): node_old.query("INSERT INTO test_replicated_merge_tree VALUES (0, 'a')") node_new.query("INSERT INTO test_replicated_merge_tree VALUES (1, 'b')") @@ -253,81 +259,116 @@ def test_replicated_merge_tree(cluster, test_case): node_old.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") node_new.query("SYSTEM SYNC REPLICA test_replicated_merge_tree") - count_old = node_old.query("SELECT count() FROM test_replicated_merge_tree").strip() - count_new = node_new.query("SELECT count() FROM test_replicated_merge_tree").strip() + count_old = node_old.query( + "SELECT count() FROM test_replicated_merge_tree" + ).strip() + count_new = node_new.query( + "SELECT count() FROM test_replicated_merge_tree" + ).strip() assert count_old == "2" assert count_new == "2" if not zero_copy: return + def get_remote_pathes(node, table_name, only_remote_path=True): - uuid = node.query(f""" + uuid = node.query( + f""" SELECT uuid FROM system.tables WHERE name = '{table_name}' - """).strip() + """ + ).strip() assert uuid - return node.query(f""" - SELECT {"remote_path" if only_remote_path else "*"} - FROM system.remote_data_paths - WHERE - local_path LIKE '%{uuid}%' - AND local_path NOT LIKE '%format_version.txt%' - ORDER BY ALL - """).strip() + return node.query( + f""" + SELECT {"remote_path" if only_remote_path else "*"} + FROM system.remote_data_paths + WHERE + local_path LIKE '%{uuid}%' + AND local_path NOT LIKE '%format_version.txt%' + ORDER BY ALL + """ + ).strip() - remote_pathes_old = get_remote_pathes(node_old, 'test_replicated_merge_tree') - remote_pathes_new = get_remote_pathes(node_new, 'test_replicated_merge_tree') + remote_pathes_old = get_remote_pathes(node_old, "test_replicated_merge_tree") + remote_pathes_new = get_remote_pathes(node_new, "test_replicated_merge_tree") assert len(remote_pathes_old) > 0 assert remote_pathes_old == remote_pathes_new, ( str(unified_diff(remote_pathes_old, remote_pathes_new)) - + "\n\nold:\n" + get_remote_pathes(node_old, 'test_replicated_merge_tree', False) - + "\n\nnew:\n" + get_remote_pathes(node_new, 'test_replicated_merge_tree', False) + + "\n\nold:\n" + + get_remote_pathes(node_old, "test_replicated_merge_tree", False) + + "\n\nnew:\n" + + get_remote_pathes(node_new, "test_replicated_merge_tree", False) ) def count_lines_with(lines, pattern): return sum([1 for x in lines if pattern in x]) - remore_pathes_with_old_format = count_lines_with(remote_pathes_old.split(), "old-style-prefix") - remore_pathes_with_new_format = count_lines_with(remote_pathes_old.split(), "new-style-prefix") + remore_pathes_with_old_format = count_lines_with( + remote_pathes_old.split(), "old-style-prefix" + ) + remore_pathes_with_new_format = count_lines_with( + remote_pathes_old.split(), "new-style-prefix" + ) if storage_policy == "s3_template_key": assert remore_pathes_with_old_format == remore_pathes_with_new_format assert remore_pathes_with_old_format == len(remote_pathes_old.split()) / 2 else: assert remore_pathes_with_old_format == len(remote_pathes_old.split()) - assert remore_pathes_with_new_format == 0 + assert remore_pathes_with_new_format == 0 - parts = node_old.query(""" - SELECT name - FROM system.parts - WHERE - table = 'test_replicated_merge_tree' - AND active - ORDER BY ALL - """).strip().split() - table_shared_uuid = node_old.query(f"SELECT value FROM system.zookeeper WHERE path='{zk_table_path}' and name='table_shared_id'").strip() + parts = ( + node_old.query( + """ + SELECT name + FROM system.parts + WHERE + table = 'test_replicated_merge_tree' + AND active + ORDER BY ALL + """ + ) + .strip() + .split() + ) + table_shared_uuid = node_old.query( + f"SELECT value FROM system.zookeeper WHERE path='{zk_table_path}' and name='table_shared_id'" + ).strip() part_blobs = {} blobs_replicas = {} for part in parts: - blobs = node_old.query(f""" - SELECT name - FROM system.zookeeper - WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}' - ORDER BY ALL - """).strip().split() + blobs = ( + node_old.query( + f""" + SELECT name + FROM system.zookeeper + WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}' + ORDER BY ALL + """ + ) + .strip() + .split() + ) for blob in blobs: - replicas = node_old.query(f""" - SELECT name - FROM system.zookeeper - WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}/{blob}' - ORDER BY ALL - """).strip().split() + replicas = ( + node_old.query( + f""" + SELECT name + FROM system.zookeeper + WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}/{blob}' + ORDER BY ALL + """ + ) + .strip() + .split() + ) assert blob not in blobs_replicas blobs_replicas[blob] = replicas @@ -345,17 +386,22 @@ def test_replicated_merge_tree(cluster, test_case): for replicas in blobs_replicas.values(): assert len(replicas) == 2, "blobs_replicas: " + str(blobs_replicas) - for blob in blobs_replicas.keys(): - assert re.match("(old-style-prefix_with-several-section|[a-z]{3}-first-random-part_new-style-prefix_constant-part)_[a-z]{3}_[a-z]{29}", blob), ( - "blobs_replicas: " + str(blobs_replicas) - ) + assert re.match( + "(old-style-prefix_with-several-section|[a-z]{3}-first-random-part_new-style-prefix_constant-part)_[a-z]{3}_[a-z]{29}", + blob, + ), "blobs_replicas: " + str(blobs_replicas) - old_style_count = sum([1 for x in blobs_replicas.keys() if "old-style-prefix" in x]) - new_style_count = sum([1 for x in blobs_replicas.keys() if "new-style-prefix" in x]) + old_style_count = sum( + [1 for x in blobs_replicas.keys() if "old-style-prefix" in x] + ) + new_style_count = sum( + [1 for x in blobs_replicas.keys() if "new-style-prefix" in x] + ) - assert (new_style_count > 0 and old_style_count == new_style_count) \ - or (new_style_count == 0 and old_style_count == len(blobs_replicas)) + assert (new_style_count > 0 and old_style_count == new_style_count) or ( + new_style_count == 0 and old_style_count == len(blobs_replicas) + ) def switch_config_write_full_object_key(node, enable): From c59a72b987c589ee6b844ee3e23e999ea6c01c1a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 3 Jan 2024 16:15:56 +0100 Subject: [PATCH 065/116] fix special build, add randomization to stateless tests --- docker/test/stateful/run.sh | 3 +++ docker/test/stateless/stress_tests.lib | 3 --- docker/test/stress/run.sh | 2 +- src/Common/MatchGenerator.cpp | 15 +++++++-------- src/Common/ObjectStorageKeyGenerator.cpp | 4 +--- tests/ci/functional_test_check.py | 1 + tests/ci/libfuzzer_test_check.py | 1 + tests/config/install.sh | 9 ++++++++- 8 files changed, 22 insertions(+), 16 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c9ce5697182..9079246429f 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -44,6 +44,9 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR # It is not needed, we will explicitly create tables on s3. # We do not have statefull tests with s3 storage run in public repository, but this is needed for another repository. rm /etc/clickhouse-server/config.d/s3_storage_policy_for_merge_tree_by_default.xml + + rm /etc/clickhouse-server/config.d/storage_metadata_with_full_object_key.xml + rm /etc/clickhouse-server/config.d/s3_storage_policy_with_template_object_key.xml fi function start() diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index e372d4db465..6f0dabb5207 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -9,9 +9,6 @@ FAIL="\tFAIL\t\\N\t" FAILURE_CONTEXT_LINES=100 FAILURE_CONTEXT_MAX_LINE_WIDTH=300 -export S3_OBJECT_KEY_TYPES=("generate-suffix" "generate-full-key" "generate-template-key") -export USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE="${S3_OBJECT_KEY_TYPES[0]}" - function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index beacd181e1d..bca8800ab2b 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -193,8 +193,8 @@ stop # Let's enable S3 storage by default export USE_S3_STORAGE_FOR_MERGE_TREE=1 +export $RANDOMIZE_OBJECT_KEY_TYPE=1 export ZOOKEEPER_FAULT_INJECTION=1 -export USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE="${S3_OBJECT_KEY_TYPES[$((RANDOM % ${#S3_OBJECT_KEY_TYPES[@]}))]}" configure # But we still need default disk because some tables loaded only into it diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 12afa0fd6e7..bc96634b518 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -188,7 +188,7 @@ private: using CharRanges = std::vector>; public: - RegexpCharClassFunction(Regexp * re_) + explicit RegexpCharClassFunction(Regexp * re_) { CharClass * cc = re_->cc(); chassert(cc); @@ -198,7 +198,7 @@ private: char_count = cc->size(); char_ranges.reserve(std::distance(cc->begin(), cc->end())); - for (auto it = cc->begin(); it != cc->end(); ++it) + for (const auto * it = cc->begin(); it != cc->end(); ++it) { char_ranges.emplace_back(it->lo, it->hi); } @@ -246,7 +246,7 @@ private: class RegexpLiteralStringFunction : public NodeFunction { public: - RegexpLiteralStringFunction(Regexp * re_) + explicit RegexpLiteralStringFunction(Regexp * re_) { if (re_->nrunes() == 0) return; @@ -279,7 +279,7 @@ private: class RegexpLiteralFunction : public NodeFunction { public: - RegexpLiteralFunction(Regexp * re_) + explicit RegexpLiteralFunction(Regexp * re_) { char buffer[UTFmax]; @@ -308,7 +308,7 @@ private: class ThrowExceptionFunction : public NodeFunction { public: - ThrowExceptionFunction(Regexp * re_) + explicit ThrowExceptionFunction(Regexp * re_) : operation(magic_enum::enum_name(re_->op())) { } @@ -332,7 +332,7 @@ private: public: - RandomStringPrepareWalker(bool logging) + explicit RandomStringPrepareWalker(bool logging) : logger(logging ? &Poco::Logger::get("GeneratorCombiner") : nullptr) { if (logger) @@ -344,7 +344,7 @@ public: if (root == nullptr) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no root has been set"); - if (generators.size() == 0) + if (generators.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no generators"); auto root_func = generators.at(root); @@ -458,7 +458,6 @@ private: return pre_arg; } -private: Poco::Logger * logger = nullptr; Regexp * root = nullptr; diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index ffac185ae3f..deae9c67e5e 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -15,7 +15,6 @@ public: , re_gen(key_template, /*logging*/ false) { } - DB::ObjectStorageKey generate(const String &) const override { return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); @@ -30,7 +29,7 @@ private: class GeneratorWithPrefix : public DB::IObjectStorageKeysGenerator { public: - GeneratorWithPrefix(String key_prefix_) + explicit GeneratorWithPrefix(String key_prefix_) : key_prefix(std::move(key_prefix_)) {} @@ -41,7 +40,6 @@ public: /// Total length is 32 a-z characters for enough randomness. /// First 3 characters are used as a prefix for /// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/ - constexpr size_t key_name_total_size = 32; constexpr size_t key_name_prefix_size = 3; diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 4d81161b6de..89fcb9ce350 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -55,6 +55,7 @@ def get_additional_envs( result.append("USE_PARALLEL_REPLICAS=1") if "s3 storage" in check_name: result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") + result.append("RANDOMIZE_OBJECT_KEY_TYPE=1") if "analyzer" in check_name: result.append("USE_NEW_ANALYZER=1") diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 6de0614541a..49699b7d2fd 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -47,6 +47,7 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total): result.append("USE_PARALLEL_REPLICAS=1") if "s3 storage" in check_name: result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") + result.append("RANDOMIZE_OBJECT_KEY_TYPE=1") if "analyzer" in check_name: result.append("USE_NEW_ANALYZER=1") diff --git a/tests/config/install.sh b/tests/config/install.sh index 8cbed155a72..2fd72245ecf 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -141,7 +141,14 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th fi if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then - case "$USE_S3_STORAGE_WITH_OBJECT_KEY_TYPE" in + object_key_types_options=("generate-suffix" "generate-full-key" "generate-template-key") + object_key_type="${object_key_types_options[$((RANDOM % ${#object_key_types_options[0]}))]}" + + if [[ -n "$RANDOMIZE_OBJECT_KEY_TYPE" ]] && [[ "$RANDOMIZE_OBJECT_KEY_TYPE" -eq 1 ]]; then + object_key_type="${randomize_options[$((RANDOM % ${#randomize_options[@]}))]}" + fi + + case object_key_type in "generate-full-key") ln -sf $SRC_PATH/config.d/storage_metadata_with_full_object_key.xml $DEST_SERVER_PATH/config.d/ ;; From 3dff1067729e017c3cf1ef800023a9ea9d6d2a4e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Jan 2024 15:35:51 +0100 Subject: [PATCH 066/116] fix clang tidy --- src/Common/MatchGenerator.cpp | 13 ++++++------- src/Common/MatchGenerator.h | 2 +- src/Common/ObjectStorageKeyGenerator.cpp | 1 - 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index bc96634b518..a2f7529cbb3 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -65,7 +64,7 @@ private: using NodeFunctionPtr = std::shared_ptr; using NodeFuncs = std::vector; - static NodeFuncs getFuncs(Children children_, const Generators & generators_) + static NodeFuncs getFuncs(const Children & children_, const Generators & generators_) { NodeFuncs result; result.reserve(children_.size()); @@ -83,7 +82,7 @@ private: class RegexpConcatFunction : public NodeFunction { public: - RegexpConcatFunction(Children children_, const Generators & generators_) + RegexpConcatFunction(const Children & children_, const Generators & generators_) : children(getFuncs(children_, generators_)) { } @@ -119,7 +118,7 @@ private: class RegexpAlternateFunction : public NodeFunction { public: - RegexpAlternateFunction(Children children_, const Generators & generators_) + RegexpAlternateFunction(const Children & children_, const Generators & generators_) : children(getFuncs(children_, generators_)) { } @@ -198,9 +197,9 @@ private: char_count = cc->size(); char_ranges.reserve(std::distance(cc->begin(), cc->end())); - for (const auto * it = cc->begin(); it != cc->end(); ++it) + for (const auto range: *cc) { - char_ranges.emplace_back(it->lo, it->hi); + char_ranges.emplace_back(range.lo, range.hi); } } @@ -475,7 +474,7 @@ void RandomStringGeneratorByRegexp::RegexpPtrDeleter::operator() (re2::Regexp * re->Decref(); } -RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(String re_str, bool logging) +RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(const String & re_str, bool logging) { re2::RE2::Options options; options.set_case_sensitive(true); diff --git a/src/Common/MatchGenerator.h b/src/Common/MatchGenerator.h index 1078c25774c..65bfe0c9208 100644 --- a/src/Common/MatchGenerator.h +++ b/src/Common/MatchGenerator.h @@ -14,7 +14,7 @@ namespace DB class RandomStringGeneratorByRegexp { public: - RandomStringGeneratorByRegexp(String re_str, bool logging); + RandomStringGeneratorByRegexp(const String & re_str, bool logging); String generate() const; private: diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index deae9c67e5e..c970b193adb 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -3,7 +3,6 @@ #include #include -#include #include From 92ad027714586624fdbadb8618f2acc759bbb6ba Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Jan 2024 15:36:06 +0100 Subject: [PATCH 067/116] add docs --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index ed413959ca6..d4251e7e74c 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1143,6 +1143,8 @@ Optional parameters: - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. - `read_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of read requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). - `write_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of write requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). +- `key_template` — Define the format with which the object keys are generated. By default, Clickhouse takes `root path` from `endpoint` option and adds random generated suffix. That suffix is a dir with 3 random symbols and a file name with 29 random symbols. With that option you have a full control how to the object keys are generated. Some usage scenarios require having random symbols in the prefix or in the middle of object key. For example: `[a-z]{3}-prefix-random/constant-part/random-middle-[a-z]{3}/random-suffix-[a-z]{29}`. The value is parsed with [`re2`](https://github.com/google/re2/wiki/Syntax). Only some subset of the syntax is supported. Check if your preferred format is supported before using that option. Disk isn't initialized if clickhouse is unable to generate a key by the value of `key_template`. It requires enabled feature flag [storage_metadata_write_full_object_key](/docs/en/operations/settings/settings#storage_metadata_write_full_object_key). It forbids declaring the `root path` in `endpoint` option. It requires definition of the option `key_compatibility_prefix`. +- `key_compatibility_prefix` — That option is required when option `key_template` is in use. In order to be able to read the objects keys which were stored in the metadata files with the metadata version lower that `VERSION_FULL_OBJECT_KEY`, the previous `root path` from the `endpoint` option should be set here. ### Configuring the cache From 4d7f455abfd1a60ef9775784b3319927451a0e9d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Jan 2024 18:11:00 +0100 Subject: [PATCH 068/116] fix bash expr --- tests/config/install.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 2fd72245ecf..73562f58b50 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -142,10 +142,10 @@ fi if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then object_key_types_options=("generate-suffix" "generate-full-key" "generate-template-key") - object_key_type="${object_key_types_options[$((RANDOM % ${#object_key_types_options[0]}))]}" + object_key_type="${object_key_types_options[0]}" if [[ -n "$RANDOMIZE_OBJECT_KEY_TYPE" ]] && [[ "$RANDOMIZE_OBJECT_KEY_TYPE" -eq 1 ]]; then - object_key_type="${randomize_options[$((RANDOM % ${#randomize_options[@]}))]}" + object_key_type="${object_key_types_options[$(($RANDOM % ${#object_key_types_options[@]}))]}" fi case object_key_type in From 66117afdd1fcf4bc3ce40889f923d2550684d4d4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 4 Jan 2024 22:45:57 +0100 Subject: [PATCH 069/116] Update install.sh --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 73562f58b50..3a0744a298d 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -148,7 +148,7 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR object_key_type="${object_key_types_options[$(($RANDOM % ${#object_key_types_options[@]}))]}" fi - case object_key_type in + case $object_key_type in "generate-full-key") ln -sf $SRC_PATH/config.d/storage_metadata_with_full_object_key.xml $DEST_SERVER_PATH/config.d/ ;; From 0d50ad29e73220e9e8bca1c083308c39285e950b Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:24:30 +0100 Subject: [PATCH 070/116] Update src/Common/MatchGenerator.cpp Co-authored-by: Antonio Andelic --- src/Common/MatchGenerator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index a2f7529cbb3..10044b59a35 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -364,7 +364,7 @@ public: } private: - Children CopyChildrenArgs(Regexp** children, int nchild) + Children CopyChildrenArgs(Regexp ** children, int nchild) { Children result; result.reserve(nchild); From e19ee6ff67f7f67c302f53ecf76c6115ad81c063 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:24:59 +0100 Subject: [PATCH 071/116] Update src/Common/MatchGenerator.cpp Co-authored-by: Antonio Andelic --- src/Common/MatchGenerator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 10044b59a35..e736cd5195c 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -377,7 +377,7 @@ private: { if (logger) LOG_DEBUG(logger, "ShortVisit"); - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "should not be call"); + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ShortVisit should not be called"); } Regexp * PreVisit(Regexp * re, Regexp* parent_arg, bool* /*stop*/) override /*noexcept*/ From 2fce803e6abf842bb78cdfd874822a73aa8b8139 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:25:14 +0100 Subject: [PATCH 072/116] Update src/Common/MatchGenerator.cpp Co-authored-by: Antonio Andelic --- src/Common/MatchGenerator.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index e736cd5195c..b6562c3aacf 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -228,8 +228,7 @@ private: auto [lo, _] = *it; Rune r = lo + count_down - 1; - int n = re2::runetochar(out, &r); - return n; + return re2::runetochar(out, &r); } size_t getRequiredSize() override From c01a6ccc8bbf62bb84acb324ba5a0c309049f668 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:25:19 +0100 Subject: [PATCH 073/116] Update src/Common/MatchGenerator.cpp Co-authored-by: Antonio Andelic --- src/Common/MatchGenerator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index b6562c3aacf..9b525bb73da 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -394,7 +394,7 @@ private: return re; } - Regexp * PostVisit(Regexp * re, Regexp* /*parent_arg*/, Regexp* pre_arg, + Regexp * PostVisit(Regexp * re, Regexp * /*parent_arg*/, Regexp * pre_arg, Regexp ** child_args, int nchild_args) override /*noexcept*/ { if (logger) From c84bad3259406ca13dc22d07a0191515bec4661d Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:25:32 +0100 Subject: [PATCH 074/116] Update src/Common/MatchGenerator.cpp Co-authored-by: Antonio Andelic --- src/Common/MatchGenerator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 9b525bb73da..5495d58d65f 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -379,7 +379,7 @@ private: throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ShortVisit should not be called"); } - Regexp * PreVisit(Regexp * re, Regexp* parent_arg, bool* /*stop*/) override /*noexcept*/ + Regexp * PreVisit(Regexp * re, Regexp * parent_arg, bool* /*stop*/) override /*noexcept*/ { if (logger) LOG_DEBUG(logger, "GeneratorCombiner PreVisit node {}", magic_enum::enum_name(re->op())); From 24fe5f4d534ed8ec2b3df36514660af08faa5367 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 5 Jan 2024 20:41:58 +0000 Subject: [PATCH 075/116] slightly better --- src/Interpreters/InterpreterSelectQuery.cpp | 36 ++++++++++----------- 1 file changed, 17 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 746f419fa1d..ed1e65a6b64 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2493,32 +2493,30 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Create optimizer with prepared actions. /// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge. - if (optimize_read_in_order || optimize_aggregation_in_order) + if (optimize_read_in_order) { - if (optimize_read_in_order) - { - query_info.order_optimizer = std::make_shared( - query, - analysis_result.order_by_elements_actions, - getSortDescription(query, context), - query_info.syntax_analyzer_result); - } - else - { - query_info.order_optimizer = std::make_shared( - query, - analysis_result.group_by_elements_actions, - getSortDescriptionFromGroupBy(query), - query_info.syntax_analyzer_result); - } + query_info.order_optimizer = std::make_shared( + query, + analysis_result.order_by_elements_actions, + getSortDescription(query, context), + query_info.syntax_analyzer_result); /// If we don't have filtration, we can pushdown limit to reading stage for optimizations. - UInt64 limit = (query.hasFiltration() || query.groupBy()) ? 0 : getLimitForSorting(query, context); + UInt64 limit = query.hasFiltration() ? 0 : getLimitForSorting(query, context); query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, limit); } + else if (optimize_aggregation_in_order) + { + query_info.order_optimizer = std::make_shared( + query, + analysis_result.group_by_elements_actions, + getSortDescriptionFromGroupBy(query), + query_info.syntax_analyzer_result); + + query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, /*limit=*/ 0); + } query_info.storage_limits = std::make_shared(storage_limits); - query_info.settings_limit_offset_done = options.settings_limit_offset_done; storage->read(query_plan, required_columns, storage_snapshot, query_info, context, processing_stage, max_block_size, max_streams); From 48a7402b34a1fac93b08827f4f590b5b3f84c54e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 6 Jan 2024 16:39:27 +0000 Subject: [PATCH 076/116] fix build --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f3917b878d6..5c068ae26df 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1398,7 +1398,7 @@ void StorageFile::read( throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", p->at(0)); auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); return; } } From d5f86f671d34a22b42300ddf13ef2bd3ad795b00 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 7 Jan 2024 02:16:29 +0100 Subject: [PATCH 077/116] fix and test that S3Clients reused --- .../table-engines/integrations/s3queue.md | 2 +- src/Common/CurrentMetrics.cpp | 2 +- src/Common/ProfileEvents.cpp | 3 + src/Coordination/KeeperSnapshotManagerS3.cpp | 2 +- src/IO/S3/Client.cpp | 23 +- src/IO/S3/Client.h | 15 +- src/IO/S3Common.cpp | 19 +- src/IO/S3Common.h | 6 +- src/Storages/StorageS3.cpp | 7 +- .../configs/defaultS3.xml | 11 - .../configs/named_collections.xml | 43 ---- .../integration/test_storage_s3_queue/test.py | 206 ++++++++++-------- 12 files changed, 163 insertions(+), 176 deletions(-) delete mode 100644 tests/integration/test_storage_s3_queue/configs/defaultS3.xml delete mode 100644 tests/integration/test_storage_s3_queue/configs/named_collections.xml diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 8b7f86cce5c..8ebab80423f 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -11,7 +11,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ``` sql CREATE TABLE s3_queue_engine_table (name String, value UInt32) - ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) + ENGINE = S3Queue(path, [NOSIGN, | aws_access_key_id, aws_secret_access_key,] format, [compression]) [SETTINGS] [mode = 'unordered',] [after_processing = 'keep',] diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2613e9ec116..724b6ab62f7 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -242,7 +242,7 @@ M(FilesystemCacheDelayedCleanupElements, "Filesystem cache elements in background cleanup queue") \ M(FilesystemCacheHoldFileSegments, "Filesystem cache file segment which are currently hold as unreleasable") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ - M(S3Requests, "S3 requests") \ + M(S3Requests, "S3 requests count") \ M(KeeperAliveConnections, "Number of alive connections") \ M(KeeperOutstandingRequets, "Number of outstanding requests") \ M(ThreadsInOvercommitTracker, "Number of waiting threads inside of OvercommitTracker") \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 119e0d99143..101959dbf83 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -391,6 +391,9 @@ The server successfully detected this situation and will download merged part fr M(DiskS3PutObject, "Number of DiskS3 API PutObject calls.") \ M(DiskS3GetObject, "Number of DiskS3 API GetObject calls.") \ \ + M(S3Clients, "Number of created S3 clients.") \ + M(TinyS3Clients, "Number of S3 clients copies which reuse an existing auth provider from another client.") \ + \ M(EngineFileLikeReadFiles, "Number of files read in table engines working with files (like File/S3/URL/HDFS).") \ \ M(ReadBufferFromS3Microseconds, "Time spent on reading from S3.") \ diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 910615bf6ef..716184e07d0 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -70,7 +70,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo { std::lock_guard client_lock{snapshot_s3_client_mutex}; // if client is not changed (same auth settings, same endpoint) we don't need to update - if (snapshot_s3_client && snapshot_s3_client->client && auth_settings == snapshot_s3_client->auth_settings + if (snapshot_s3_client && snapshot_s3_client->client && !snapshot_s3_client->auth_settings.hasUpdates(auth_settings) && snapshot_s3_client->uri.uri == new_uri.uri) return; } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index a65a82d9b40..b65de8d34a7 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -3,7 +3,6 @@ #if USE_AWS_S3 #include -#include #include #include #include @@ -15,7 +14,6 @@ #include -#include #include #include #include @@ -37,6 +35,9 @@ namespace ProfileEvents extern const Event DiskS3WriteRequestsErrors; extern const Event DiskS3ReadRequestsErrors; + + extern const Event S3Clients; + extern const Event TinyS3Clients; } namespace DB @@ -199,6 +200,8 @@ Client::Client( cache = std::make_shared(); ClientCacheRegistry::instance().registerClient(cache); + + ProfileEvents::increment(ProfileEvents::S3Clients); } Client::Client( @@ -219,6 +222,22 @@ Client::Client( { cache = std::make_shared(*other.cache); ClientCacheRegistry::instance().registerClient(cache); + + ProfileEvents::increment(ProfileEvents::TinyS3Clients); +} + + +Client::~Client() +{ + try + { + ClientCacheRegistry::instance().unregisterClient(cache.get()); + } + catch (...) + { + tryLogCurrentException(log); + throw; + } } Aws::Auth::AWSCredentials Client::getCredentials() const diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index b137f0605dc..677b739fd39 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -142,18 +142,7 @@ public: Client(Client && other) = delete; Client & operator=(Client &&) = delete; - ~Client() override - { - try - { - ClientCacheRegistry::instance().unregisterClient(cache.get()); - } - catch (...) - { - tryLogCurrentException(log); - throw; - } - } + ~Client() override; /// Returns the initial endpoint. const String & getInitialEndpoint() const { return initial_endpoint; } @@ -170,7 +159,7 @@ public: class RetryStrategy : public Aws::Client::RetryStrategy { public: - RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000); + explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000); /// NOLINTNEXTLINE(google-runtime-int) bool ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 96ad6413ef5..f2ead22127c 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -6,21 +6,12 @@ #if USE_AWS_S3 -# include - -# include # include -# include - -# include -# include # include -# include # include -# include +# include # include -# include namespace ProfileEvents { @@ -147,6 +138,12 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const }; } +bool AuthSettings::hasUpdates(const AuthSettings & other) const +{ + AuthSettings copy = *this; + copy.updateFrom(other); + return copy != other; +} void AuthSettings::updateFrom(const AuthSettings & from) { @@ -175,7 +172,7 @@ void AuthSettings::updateFrom(const AuthSettings & from) expiration_window_seconds = from.expiration_window_seconds; if (from.no_sign_request.has_value()) - no_sign_request = *from.no_sign_request; + no_sign_request = from.no_sign_request; } } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index ebfc07a3976..6ee8d96ed09 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -92,9 +92,11 @@ struct AuthSettings std::optional expiration_window_seconds; std::optional no_sign_request; - bool operator==(const AuthSettings & other) const = default; - + bool hasUpdates(const AuthSettings & other) const; void updateFrom(const AuthSettings & from); + +private: + bool operator==(const AuthSettings & other) const = default; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 60ae7f219f4..0909539033d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include #include @@ -1427,7 +1429,7 @@ bool StorageS3::Configuration::update(ContextPtr context) request_settings = s3_settings.request_settings; request_settings.updateFromSettings(context->getSettings()); - if (client && (static_configuration || s3_settings.auth_settings == auth_settings)) + if (client && (static_configuration || !s3_settings.auth_settings.hasUpdates(auth_settings))) return false; auth_settings.updateFrom(s3_settings.auth_settings); @@ -1650,11 +1652,10 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context if (engine_args_to_idx.contains("session_token")) configuration.auth_settings.session_token = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["session_token"]], "session_token"); - configuration.auth_settings.no_sign_request = no_sign_request; } - configuration.static_configuration = !configuration.auth_settings.access_key_id.empty(); + configuration.static_configuration = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request; configuration.keys = {configuration.url.key}; diff --git a/tests/integration/test_storage_s3_queue/configs/defaultS3.xml b/tests/integration/test_storage_s3_queue/configs/defaultS3.xml deleted file mode 100644 index 7dac6d9fbb5..00000000000 --- a/tests/integration/test_storage_s3_queue/configs/defaultS3.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - http://resolver:8080 -
Authorization: Bearer TOKEN
-
- - http://resolver:8080/root-with-auth/restricteddirectory/ - -
-
diff --git a/tests/integration/test_storage_s3_queue/configs/named_collections.xml b/tests/integration/test_storage_s3_queue/configs/named_collections.xml deleted file mode 100644 index 64674e2a3e3..00000000000 --- a/tests/integration/test_storage_s3_queue/configs/named_collections.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - http://minio1:9001/root/test_table - minio - minio123 - - - http://minio1:9001/root/test_parquet - minio - minio123 - - - http://minio1:9001/root/test_parquet_gz - minio - minio123 - - - http://minio1:9001/root/test_orc - minio - minio123 - - - http://minio1:9001/root/test_native - minio - minio123 - - - http://minio1:9001/root/test.arrow - minio - minio123 - - - http://minio1:9001/root/test.parquet - minio - minio123 - - - http://minio1:9001/root/test_cache4.jsonl - true - - - diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b83c095a7a6..6fee339c29b 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1,6 +1,5 @@ import io import logging -import os import random import time @@ -9,75 +8,59 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance import json -""" -export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-server -export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-client -export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-odbc-bridge -export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/server -""" - -MINIO_INTERNAL_PORT = 9001 AVAILABLE_MODES = ["unordered", "ordered"] -AUTH = "'minio','minio123'," -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_AUTH = ["'minio'", "'minio123'"] +NO_AUTH = ["NOSIGN"] -def prepare_s3_bucket(started_cluster): - # Allows read-write access for bucket without authorization. - bucket_read_write_policy = { - "Version": "2012-10-17", - "Statement": [ - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:GetBucketLocation", - "Resource": "arn:aws:s3:::root", - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:ListBucket", - "Resource": "arn:aws:s3:::root", - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:GetObject", - "Resource": "arn:aws:s3:::root/*", - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:PutObject", - "Resource": "arn:aws:s3:::root/*", - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:DeleteObject", - "Resource": "arn:aws:s3:::root/*", - }, - ], - } +def prepare_public_s3_bucket(started_cluster): + def create_bucket(client, bucket_name, policy): + if client.bucket_exists(bucket_name): + client.remove_bucket(bucket_name) + + client.make_bucket(bucket_name) + + client.set_bucket_policy( + bucket_name, json.dumps(policy) + ) + + def get_policy_with_public_access(bucket_name): + return { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": "*", + "Action": [ + "s3:GetBucketLocation", + "s3:ListBucket", + ], + "Resource": f"arn:aws:s3:::{bucket_name}", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": "*", + "Action": [ + "s3:GetObject", + "s3:PutObject", + "s3:DeleteObject", + ], + "Resource": f"arn:aws:s3:::{bucket_name}/*", + }, + ], + } minio_client = started_cluster.minio_client - minio_client.set_bucket_policy( - started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) - ) - started_cluster.minio_restricted_bucket = "{}-with-auth".format( - started_cluster.minio_bucket + started_cluster.minio_public_bucket = f"{started_cluster.minio_bucket}-public" + create_bucket( + minio_client, + started_cluster.minio_public_bucket, + get_policy_with_public_access(started_cluster.minio_public_bucket), ) - if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): - minio_client.remove_bucket(started_cluster.minio_restricted_bucket) - - minio_client.make_bucket(started_cluster.minio_restricted_bucket) @pytest.fixture(autouse=True) @@ -90,10 +73,10 @@ def s3_queue_setup_teardown(started_cluster): minio = started_cluster.minio_client objects = list( - minio.list_objects(started_cluster.minio_restricted_bucket, recursive=True) + minio.list_objects(started_cluster.minio_bucket, recursive=True) ) for obj in objects: - minio.remove_object(started_cluster.minio_restricted_bucket, obj.object_name) + minio.remove_object(started_cluster.minio_bucket, obj.object_name) yield # run test @@ -107,8 +90,6 @@ def started_cluster(): with_minio=True, with_zookeeper=True, main_configs=[ - "configs/defaultS3.xml", - "configs/named_collections.xml", "configs/zookeeper.xml", "configs/s3queue_log.xml", ], @@ -119,8 +100,6 @@ def started_cluster(): with_minio=True, with_zookeeper=True, main_configs=[ - "configs/defaultS3.xml", - "configs/named_collections.xml", "configs/s3queue_log.xml", ], ) @@ -129,7 +108,6 @@ def started_cluster(): cluster.start() logging.info("Cluster started") - prepare_s3_bucket(cluster) yield cluster finally: cluster.shutdown() @@ -146,7 +124,7 @@ def run_query(instance, query, stdin=None, settings=None): def generate_random_files( - started_cluster, files_path, count, column_num=3, row_num=10, start_ind=0 + started_cluster, files_path, count, column_num=3, row_num=10, start_ind=0, bucket=None ): files = [ (f"{files_path}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) @@ -164,30 +142,18 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - put_s3_file_content(started_cluster, filename, values_csv) + put_s3_file_content(started_cluster, filename, values_csv, bucket) return total_values -def put_s3_file_content(started_cluster, filename, data): +def put_s3_file_content(started_cluster, filename, data, bucket=None): + bucket = started_cluster.minio_bucket if bucket is None else bucket buf = io.BytesIO(data) started_cluster.minio_client.put_object( - started_cluster.minio_bucket, filename, buf, len(data) + bucket, filename, buf, len(data) ) -def get_s3_file_content(started_cluster, bucket, filename, decode=True): - # type: (ClickHouseCluster, str, str, bool) -> str - # Returns content of given S3 file as string. - - data = started_cluster.minio_client.get_object(bucket, filename) - data_str = b"" - for chunk in data.stream(): - data_str += chunk - if decode: - return data_str.decode() - return data_str - - def create_table( started_cluster, node, @@ -197,7 +163,12 @@ def create_table( format="column1 UInt32, column2 UInt32, column3 UInt32", additional_settings={}, file_format="CSV", + auth=DEFAULT_AUTH, + bucket=None, ): + auth_params = ",".join(auth) + bucket = started_cluster.minio_bucket if bucket is None else bucket + settings = { "s3queue_loading_retries": 0, "after_processing": "keep", @@ -206,11 +177,11 @@ def create_table( } settings.update(additional_settings) - url = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{started_cluster.minio_bucket}/{files_path}/" + url = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/" node.query(f"DROP TABLE IF EXISTS {table_name}") create_query = f""" CREATE TABLE {table_name} ({format}) - ENGINE = S3Queue('{url}', {AUTH}'{file_format}') + ENGINE = S3Queue('{url}', {auth_params}, {file_format}) SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))} """ node.query(create_query) @@ -922,3 +893,62 @@ def test_drop_table(started_cluster): ) or node.contains_in_log( f"StorageS3Queue ({table_name}): Shutdown was called, stopping sync" ) + + +def test_s3_client_reused(started_cluster): + node = started_cluster.instances["instance"] + table_name = f"test.test_s3_client_reused" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" + row_num = 10 + + def get_created_s3_clients_count(): + value = node.query(f"SELECT value FROM system.events WHERE event='S3Clients'").strip() + return int(value) if value != '' else 0 + + + def wait_all_processed(files_num): + expected_count = files_num * row_num + for _ in range(100): + count = int(node.query(f"SELECT count() FROM {dst_table_name}")) + print(f"{count}/{expected_count}") + if count == expected_count: + break + time.sleep(1) + assert int(node.query(f"SELECT count() FROM {dst_table_name}")) == expected_count + + prepare_public_s3_bucket(started_cluster) + + s3_clients_before = get_created_s3_clients_count() + + create_table( + started_cluster, + node, + table_name, + "ordered", + files_path, + additional_settings={ + "after_processing": "delete", + "s3queue_processing_threads_num": 1, + }, + auth=NO_AUTH, + bucket=started_cluster.minio_public_bucket, + ) + + s3_clients_after = get_created_s3_clients_count() + assert s3_clients_before + 1 == s3_clients_after + + create_mv(node, table_name, dst_table_name) + + for i in range(0, 10): + s3_clients_before = get_created_s3_clients_count() + + generate_random_files( + started_cluster, files_path, count=1, start_ind=i, row_num=row_num, bucket=started_cluster.minio_public_bucket + ) + + wait_all_processed(i+1) + + s3_clients_after = get_created_s3_clients_count() + + assert s3_clients_before == s3_clients_after From 43896784a6f04a50a44e6ffe0054aaa6a28d4313 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 18:00:41 +0000 Subject: [PATCH 078/116] Cosmetics --- src/Functions/FunctionsStringDistance.cpp | 12 ++-- .../02884_string_distance_function.sql | 56 +++++++++---------- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index a5e819179d6..d563d719ac5 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -25,7 +25,7 @@ struct FunctionStringDistanceImpl { using ResultType = typename Op::ResultType; - static void constantConstant(const std::string & haystack, const std::string & needle, ResultType & res) + static void constantConstant(const String & haystack, const String & needle, ResultType & res) { res = Op::process(haystack.data(), haystack.size(), needle.data(), needle.size()); } @@ -51,7 +51,7 @@ struct FunctionStringDistanceImpl } static void constantVector( - const std::string & haystack, + const String & haystack, const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, PaddedPODArray & res) @@ -70,7 +70,7 @@ struct FunctionStringDistanceImpl static void vectorConstant( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - const std::string & needle, + const String & needle, PaddedPODArray & res) { constantVector(needle, data, offsets, res); @@ -81,7 +81,7 @@ struct FunctionStringDistanceImpl struct ByteHammingDistanceImpl { using ResultType = UInt64; - static ResultType inline process( + static ResultType process( const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) { UInt64 res = 0; @@ -115,7 +115,7 @@ template struct ByteJaccardIndexImpl { using ResultType = Float64; - static ResultType inline process( + static ResultType process( const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) { if (haystack_size == 0 || needle_size == 0) @@ -227,7 +227,7 @@ struct ByteEditDistanceImpl using ResultType = UInt64; static constexpr size_t max_string_size = 1u << 16; - static ResultType inline process( + static ResultType process( const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) { if (haystack_size == 0 || needle_size == 0) diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index e3d9051ce5b..d2a6d9c2163 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -1,34 +1,34 @@ -select 'const arguments byteHammingDistance'; -select byteHammingDistance('abcd', 'abcd'); -select 'const arguments editDistance'; -select editDistance('clickhouse', 'mouse'); +SELECT 'const arguments byteHammingDistance'; +SELECT byteHammingDistance('abcd', 'abcd'); +SELECT 'const arguments editDistance'; +SELECT editDistance('clickhouse', 'mouse'); -select 'const arguments stringJaccardIndex'; -select stringJaccardIndex('clickhouse', 'mouse'); +SELECT 'const arguments stringJaccardIndex'; +SELECT stringJaccardIndex('clickhouse', 'mouse'); -drop table if exists t; -create table t +DROP TABLE if exists t; +CREATE TABLE t ( s1 String, s2 String -) engine = MergeTree order by s1; +) ENGINE = MergeTree ORDER BY s1; -insert into t values ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '') ('mouse', 'clickhouse'); -select 'byteHammingDistance'; -select byteHammingDistance(s1, s2) FROM t ORDER BY s1, s2; -select 'byteHammingDistance(const, non const)'; -select byteHammingDistance('abc', s2) FROM t ORDER BY s1, s2; -select 'byteHammingDistance(non const, const)'; -select byteHammingDistance(s2, 'def') FROM t ORDER BY s1, s2; +INSERT INTO t VALUES ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '') ('mouse', 'clickhouse'); +SELECT 'byteHammingDistance'; +SELECT byteHammingDistance(s1, s2) FROM t ORDER BY s1, s2; +SELECT 'byteHammingDistance(const, non const)'; +SELECT byteHammingDistance('abc', s2) FROM t ORDER BY s1, s2; +SELECT 'byteHammingDistance(non const, const)'; +SELECT byteHammingDistance(s2, 'def') FROM t ORDER BY s1, s2; -select 'mismatches(alias)'; -select mismatches(s1, s2) FROM t ORDER BY s1, s2; -select mismatches('abc', s2) FROM t ORDER BY s1, s2; -select mismatches(s2, 'def') FROM t ORDER BY s1, s2; +SELECT 'mismatches(alias)'; +SELECT mismatches(s1, s2) FROM t ORDER BY s1, s2; +SELECT mismatches('abc', s2) FROM t ORDER BY s1, s2; +SELECT mismatches(s2, 'def') FROM t ORDER BY s1, s2; -select 'stringJaccardIndex'; -select stringJaccardIndex(s1, s2) FROM t ORDER BY s1, s2; -select stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY s1, s2; +SELECT 'stringJaccardIndex'; +SELECT stringJaccardIndex(s1, s2) FROM t ORDER BY s1, s2; +SELECT stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY s1, s2; -- we do not perform full UTF8 validation, so sometimes it just returns some result SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\x48\x65\x6C')); @@ -45,11 +45,11 @@ SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xDC\x00')); -- SELECT stringJaccardIndexUTF8('😃🌍', '🙃😃🌑'), stringJaccardIndex('😃🌍', '🙃😃🌑'); -select 'editDistance'; -select editDistance(s1, s2) FROM t ORDER BY s1, s2; -select 'levenshteinDistance'; -select levenshteinDistance(s1, s2) FROM t ORDER BY s1, s2; +SELECT 'editDistance'; +SELECT editDistance(s1, s2) FROM t ORDER BY s1, s2; +SELECT 'levenshteinDistance'; +SELECT levenshteinDistance(s1, s2) FROM t ORDER BY s1, s2; SELECT editDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} -drop table t; +DROP TABLE t; From 7d98fdede741910417d92ce2a56a4a08e561490d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 19:42:18 +0000 Subject: [PATCH 079/116] Implement Dramerau-Levenshtein distance --- .../functions/string-functions.md | 24 +++++ src/Functions/FunctionsStringDistance.cpp | 79 +++++++++++++- .../02884_string_distance_function.reference | 100 ++++++++---------- .../02884_string_distance_function.sql | 60 +++++------ 4 files changed, 174 insertions(+), 89 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 20694211912..c600de13138 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1463,6 +1463,30 @@ Result: Alias: levenshteinDistance +## damerauLevenshteinDistance + +Calculates the [Damerau-Levenshtein distance](https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance) between two byte strings. + +**Syntax** + +```sql +damerauLevenshteinDistance(string1, string2) +``` + +**Examples** + +``` sql +SELECT damerauLevenshteinDistance('clickhouse', 'mouse'); +``` + +Result: + +``` text +┌─damerauLevenshteinDistance('clickhouse', 'mouse')─┐ +│ 6 │ +└───────────────────────────────────────────────────┘ +``` + ## initcap Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index d563d719ac5..4fe7446fa85 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -8,6 +8,8 @@ #include #include +#include + #ifdef __SSE4_2__ # include #endif @@ -222,10 +224,11 @@ struct ByteJaccardIndexImpl } }; +static constexpr size_t max_string_size = 1u << 16; + struct ByteEditDistanceImpl { using ResultType = UInt64; - static constexpr size_t max_string_size = 1u << 16; static ResultType process( const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) @@ -237,8 +240,7 @@ struct ByteEditDistanceImpl if (haystack_size > max_string_size || needle_size > max_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, - "The string size is too big for function editDistance, " - "should be at most {}", max_string_size); + "The string size is too big for function editDistance, should be at most {}", max_string_size); PaddedPODArray distances0(haystack_size + 1, 0); PaddedPODArray distances1(haystack_size + 1, 0); @@ -271,6 +273,68 @@ struct ByteEditDistanceImpl } }; +struct ByteDamerauLevenshteinDistanceImpl +{ + using ResultType = UInt64; + + static ResultType process( + const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) + { + /// Safety threshold against DoS, since we use two array to calculate the distance. + if (haystack_size > max_string_size || needle_size > max_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "The string size is too big for function damerauLevenshteinDistance, should be at most {}", max_string_size); + + /// Shortcuts: + + if (haystack_size == 0) + return needle_size; + + if (needle_size == 0) + return haystack_size; + + if (haystack_size == needle_size && memcmp(haystack, needle, haystack_size) == 0) + return 0; + + /// Implements the algorithm for optimal string alignment distance: + + /// Dynamically allocate memory for the 2D array + /// Allocating a 2D array, for convenience starts is an array of pointers to the start of the rows. + std::vector rows((needle_size + 1) * (haystack_size + 1)); + std::vector starts(haystack_size + 1); + + /// Setting the pointers in starts to the beginning of (needle_size + 1)-long intervals. + /// Also initialize the row values based on the mentioned algorithm. + for (size_t i = 0; i <= haystack_size; ++i) + { + starts[i] = rows.data() + (needle_size + 1) * i; + starts[i][0] = static_cast(i); + } + + for (size_t j = 0; j <= needle_size; ++j) + { + starts[0][j] = static_cast(j); + } + + for (size_t i = 1; i <= haystack_size; ++i) + { + for (size_t j = 1; j <= needle_size; ++j) + { + int cost = (haystack[i - 1] == needle[j - 1]) ? 0 : 1; + starts[i][j] = std::min(starts[i - 1][j - 1] + cost, /// substitution + std::min(starts[i][j - 1] + 1, /// insertion + starts[i - 1][j] + 1) /// deletion + ); + if (i > 1 && j > 1 && haystack[i - 1] == needle[j - 2] && haystack[i - 2] == needle[j - 1]) + starts[i][j] = std::min(starts[i][j], starts[i - 2][j - 2] + 1); /// transposition + } + } + + return starts[haystack_size][needle_size]; + } +}; + struct NameByteHammingDistance { static constexpr auto name = "byteHammingDistance"; @@ -283,6 +347,12 @@ struct NameEditDistance }; using FunctionEditDistance = FunctionsStringSimilarity, NameEditDistance>; +struct NameDamerauLevenshteinDistance +{ + static constexpr auto name = "damerauLevenshteinDistance"; +}; +using FunctionDamerauLevenshteinDistance = FunctionsStringSimilarity, NameDamerauLevenshteinDistance>; + struct NameJaccardIndex { static constexpr auto name = "stringJaccardIndex"; @@ -305,6 +375,9 @@ REGISTER_FUNCTION(StringDistance) FunctionDocumentation{.description = R"(Calculates the edit distance between two byte-strings.)"}); factory.registerAlias("levenshteinDistance", NameEditDistance::name); + factory.registerFunction( + FunctionDocumentation{.description = R"(Calculates the Damerau-Levenshtein distance two between two byte-string.)"}); + factory.registerFunction( FunctionDocumentation{.description = R"(Calculates the [Jaccard similarity index](https://en.wikipedia.org/wiki/Jaccard_index) between two byte strings.)"}); factory.registerFunction( diff --git a/tests/queries/0_stateless/02884_string_distance_function.reference b/tests/queries/0_stateless/02884_string_distance_function.reference index 3ac30825fd0..0b370a43483 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.reference +++ b/tests/queries/0_stateless/02884_string_distance_function.reference @@ -1,59 +1,53 @@ -const arguments byteHammingDistance -0 -const arguments editDistance -6 -const arguments stringJaccardIndex -0.4 -byteHammingDistance -7 -1 -7 -10 -byteHammingDistance(const, non const) -3 -3 -6 -10 -byteHammingDistance(non const, const) -3 -6 -6 -10 -mismatches(alias) -7 -1 -7 -10 -3 -3 -6 -10 -3 -6 -6 -10 -stringJaccardIndex -0 -0.8571428571428571 -0.8571428571428571 -0.4 -0 -0.8571428571428571 -0.8571428571428571 -0.4 +-- const arguments +clickhouse mouse 10 +clickhouse mouse 6 +clickhouse mouse 6 +clickhouse mouse 0.4 +clickhouse mouse 0.4 +-- test aliases +clickhouse mouse 10 +clickhouse mouse 6 +-- Deny DoS using too large inputs +-- non-const arguments +byteHammingDistance 0 +byteHammingDistance abc 3 +byteHammingDistance abc 3 +byteHammingDistance abc ab 1 +byteHammingDistance abc abc 0 +byteHammingDistance abc bc 3 +byteHammingDistance clickhouse mouse 10 +editDistance 0 +editDistance abc 3 +editDistance abc 3 +editDistance abc ab 1 +editDistance abc abc 0 +editDistance abc bc 1 +editDistance clickhouse mouse 6 +damerauLevenshteinDistance 0 +damerauLevenshteinDistance abc 3 +damerauLevenshteinDistance abc 3 +damerauLevenshteinDistance abc ab 1 +damerauLevenshteinDistance abc abc 0 +damerauLevenshteinDistance abc bc 1 +damerauLevenshteinDistance clickhouse mouse 6 +stringJaccardIndex 0 +stringJaccardIndex abc 0 +stringJaccardIndex abc 0 +stringJaccardIndex abc ab 0.6666666666666666 +stringJaccardIndex abc abc 1 +stringJaccardIndex abc bc 0.6666666666666666 +stringJaccardIndex clickhouse mouse 0.4 +stringJaccardIndexUTF8 0 +stringJaccardIndexUTF8 abc 0 +stringJaccardIndexUTF8 abc 0 +stringJaccardIndexUTF8 abc ab 0.6666666666666666 +stringJaccardIndexUTF8 abc abc 1 +stringJaccardIndexUTF8 abc bc 0.6666666666666666 +stringJaccardIndexUTF8 clickhouse mouse 0.4 +-- Special UTF-8 tests 0.4 0 0 0 0 0.25 0.625 -editDistance -7 -1 -1 -6 -levenshteinDistance -7 -1 -1 -6 diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index d2a6d9c2163..c9c8f949249 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -1,36 +1,38 @@ -SELECT 'const arguments byteHammingDistance'; -SELECT byteHammingDistance('abcd', 'abcd'); -SELECT 'const arguments editDistance'; -SELECT editDistance('clickhouse', 'mouse'); +SELECT '-- const arguments'; +-- just to see it works +SELECT 'clickhouse' AS s1, 'mouse' AS s2, byteHammingDistance(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, editDistance(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, damerauLevenshteinDistance(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, stringJaccardIndex(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, stringJaccardIndexUTF8(s1, s2); -SELECT 'const arguments stringJaccardIndex'; -SELECT stringJaccardIndex('clickhouse', 'mouse'); +SELECT '-- test aliases'; +SELECT 'clickhouse' AS s1, 'mouse' AS s2, mismatches(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, levenshteinDistance(s1, s2); -DROP TABLE if exists t; +SELECT '-- Deny DoS using too large inputs'; +SELECT editDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} +SELECT damerauLevenshteinDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} + +DROP TABLE IF EXISTS t; CREATE TABLE t ( - s1 String, - s2 String + s1 String, + s2 String ) ENGINE = MergeTree ORDER BY s1; -INSERT INTO t VALUES ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '') ('mouse', 'clickhouse'); -SELECT 'byteHammingDistance'; -SELECT byteHammingDistance(s1, s2) FROM t ORDER BY s1, s2; -SELECT 'byteHammingDistance(const, non const)'; -SELECT byteHammingDistance('abc', s2) FROM t ORDER BY s1, s2; -SELECT 'byteHammingDistance(non const, const)'; -SELECT byteHammingDistance(s2, 'def') FROM t ORDER BY s1, s2; +-- actual test cases +INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse'); -SELECT 'mismatches(alias)'; -SELECT mismatches(s1, s2) FROM t ORDER BY s1, s2; -SELECT mismatches('abc', s2) FROM t ORDER BY s1, s2; -SELECT mismatches(s2, 'def') FROM t ORDER BY s1, s2; +SELECT '-- non-const arguments'; +SELECT 'byteHammingDistance', s1, s2, byteHammingDistance(s1, s2) FROM t ORDER BY ALL; +SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY ALL; +SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY ALL; +SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY ALL; +SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY ALL; -SELECT 'stringJaccardIndex'; -SELECT stringJaccardIndex(s1, s2) FROM t ORDER BY s1, s2; -SELECT stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY s1, s2; - --- we do not perform full UTF8 validation, so sometimes it just returns some result +SELECT '-- Special UTF-8 tests'; +-- We do not perform full UTF8 validation, so sometimes it just returns some result SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\x48\x65\x6C')); SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xFF\xFF\xFF\xFF')); SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\x41\xE2\x82\xAC')); @@ -42,14 +44,6 @@ SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xF0\x80\x80\x4 SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xC0\x80')); -- { serverError BAD_ARGUMENTS } SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xD8\x00 ')); -- { serverError BAD_ARGUMENTS } SELECT stringJaccardIndexUTF8(materialize('hello'), materialize('\xDC\x00')); -- { serverError BAD_ARGUMENTS } - SELECT stringJaccardIndexUTF8('😃🌍', '🙃😃🌑'), stringJaccardIndex('😃🌍', '🙃😃🌑'); -SELECT 'editDistance'; -SELECT editDistance(s1, s2) FROM t ORDER BY s1, s2; -SELECT 'levenshteinDistance'; -SELECT levenshteinDistance(s1, s2) FROM t ORDER BY s1, s2; - -SELECT editDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} - DROP TABLE t; From ee0738e3df23c1934326692e868e34dac9b05102 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 20:14:35 +0000 Subject: [PATCH 080/116] Implement jaroSimilarity + jaroWinklerSimilarity --- .../functions/string-functions.md | 48 ++++++ src/Functions/FunctionsStringDistance.cpp | 140 +++++++++++++++++- .../02884_string_distance_function.reference | 16 ++ .../02884_string_distance_function.sql | 4 + 4 files changed, 203 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c600de13138..88faba90c92 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1487,6 +1487,54 @@ Result: └───────────────────────────────────────────────────┘ ``` +## jaroSimilarity + +Calculates the [Jaro similarity](https://en.wikipedia.org/wiki/Jaro%E2%80%93Winkler_distance#Jaro_similarity) between two byte strings. + +**Syntax** + +```sql +jaroSimilarity(string1, string2) +``` + +**Examples** + +``` sql +SELECT jaroSimilarity('clickhouse', 'click'); +``` + +Result: + +``` text +┌─jaroSimilarity('clickhouse', 'click')─┐ +│ 0.8333333333333333 │ +└───────────────────────────────────────┘ +``` + +## jaroWinklerSimilarity + +Calculates the [Jaro-Winkler similarity](https://en.wikipedia.org/wiki/Jaro%E2%80%93Winkler_distance#Jaro%E2%80%93Winkler_similarity) between two byte strings. + +**Syntax** + +```sql +jaroWinklerSimilarity(string1, string2) +``` + +**Examples** + +``` sql +SELECT jaroWinklerSimilarity('clickhouse', 'click'); +``` + +Result: + +``` text +┌─jaroWinklerSimilarity('clickhouse', 'click')─┐ +│ 0.8999999999999999 │ +└──────────────────────────────────────────────┘ +``` + ## initcap Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 4fe7446fa85..86f5a48b7b8 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -236,7 +236,7 @@ struct ByteEditDistanceImpl if (haystack_size == 0 || needle_size == 0) return haystack_size + needle_size; - /// Safety threshold against DoS, since we use two array to calculate the distance. + /// Safety threshold against DoS, since we use two arrays to calculate the distance. if (haystack_size > max_string_size || needle_size > max_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, @@ -280,7 +280,7 @@ struct ByteDamerauLevenshteinDistanceImpl static ResultType process( const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) { - /// Safety threshold against DoS, since we use two array to calculate the distance. + /// Safety threshold against DoS if (haystack_size > max_string_size || needle_size > max_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, @@ -335,6 +335,118 @@ struct ByteDamerauLevenshteinDistanceImpl } }; +struct ByteJaroSimilarityImpl { + + using ResultType = Float64; + + static ResultType process( + const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) + { + /// Safety threshold against DoS + if (haystack_size > max_string_size || needle_size > max_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "The string size is too big for function jaroSimilarity, should be at most {}", max_string_size); + + /// Shortcuts: + + if (haystack_size == 0) + return needle_size; + + if (needle_size == 0) + return haystack_size; + + if (haystack_size == needle_size && memcmp(haystack, needle, haystack_size) == 0) + return 1.0; + + const int s1len = static_cast(haystack_size); + const int s2len = static_cast(needle_size); + + /// Window size to search for matches in the other string + const int max_range = std::max(0, std::max(s1len, s2len) / 2 - 1); + std::vector s1_matching(s1len, -1); + std::vector s2_matching(s2len, -1); + + /// Calculate matching characters + size_t matching_characters = 0; + for (int i = 0; i < s1len; i++) + { + /// Matching window + const int min_index = std::max(i - max_range, 0); + const int max_index = std::min(i + max_range + 1, s2len); + for (int j = min_index; j < max_index; j++) + { + if (s2_matching[j] == -1 && haystack[i] == needle[j]) { + s1_matching[i] = i; + s2_matching[j] = j; + matching_characters++; + break; + } + } + } + + if (matching_characters == 0) + return 0.0; + + /// Transpositions (one-way only) + double transpositions = 0.0; + for (size_t i = 0, s1i = 0, s2i = 0; i < matching_characters; i++) + { + while (s1_matching[s1i] == -1) + s1i++; + while (s2_matching[s2i] == -1) + s2i++; + if (haystack[s1i] != needle[s2i]) + transpositions += 0.5; + s1i++; + s2i++; + } + double m = static_cast(matching_characters); + double jaro_similarity = 1.0 / 3.0 * ( m / static_cast(s1len) + + m / static_cast(s2len) + + (m - transpositions) / m ); + return jaro_similarity; + } +}; + +struct ByteJaroWinklerSimilarityImpl { + + using ResultType = Float64; + + static ResultType process( + const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size) + { + static constexpr int max_prefix_length = 4; + static constexpr double scaling_factor = 0.1; + static constexpr double boost_threshold = 0.7; + + /// Safety threshold against DoS + if (haystack_size > max_string_size || needle_size > max_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "The string size is too big for function jaroWinklerSimilarity, should be at most {}", max_string_size); + + const int s1len = static_cast(haystack_size); + const int s2len = static_cast(needle_size); + + ResultType jaro_winkler_similarity = ByteJaroSimilarityImpl::process(haystack, haystack_size, needle, needle_size); + + if (jaro_winkler_similarity== -1.0) + return -1.0; + + if (jaro_winkler_similarity> boost_threshold) + { + const int common_length = std::min(max_prefix_length, std::min(s1len, s2len)); + int common_prefix = 0; + while (common_prefix < common_length && haystack[common_prefix] == needle[common_prefix]) + common_prefix++; + + jaro_winkler_similarity += common_prefix * scaling_factor * (1.0 - jaro_winkler_similarity); + } + return jaro_winkler_similarity; + } +}; + struct NameByteHammingDistance { static constexpr auto name = "byteHammingDistance"; @@ -365,6 +477,18 @@ struct NameJaccardIndexUTF8 }; using FunctionStringJaccardIndexUTF8 = FunctionsStringSimilarity>, NameJaccardIndexUTF8>; +struct NameJaroSimilarity +{ + static constexpr auto name = "jaroSimilarity"; +}; +using FunctionJaroSimilarity = FunctionsStringSimilarity, NameJaroSimilarity>; + +struct NameJaroWinklerSimilarity +{ + static constexpr auto name = "jaroWinklerSimilarity"; +}; +using FunctionJaroWinklerSimilarity = FunctionsStringSimilarity, NameJaroWinklerSimilarity>; + REGISTER_FUNCTION(StringDistance) { factory.registerFunction( @@ -376,11 +500,17 @@ REGISTER_FUNCTION(StringDistance) factory.registerAlias("levenshteinDistance", NameEditDistance::name); factory.registerFunction( - FunctionDocumentation{.description = R"(Calculates the Damerau-Levenshtein distance two between two byte-string.)"}); + FunctionDocumentation{.description = R"(Calculates the Damerau-Levenshtein distance two between two byte-string.)"}); factory.registerFunction( - FunctionDocumentation{.description = R"(Calculates the [Jaccard similarity index](https://en.wikipedia.org/wiki/Jaccard_index) between two byte strings.)"}); + FunctionDocumentation{.description = R"(Calculates the Jaccard similarity index between two byte strings.)"}); factory.registerFunction( - FunctionDocumentation{.description = R"(Calculates the [Jaccard similarity index](https://en.wikipedia.org/wiki/Jaccard_index) between two UTF8 strings.)"}); + FunctionDocumentation{.description = R"(Calculates the Jaccard similarity index between two UTF8 strings.)"}); + + factory.registerFunction( + FunctionDocumentation{.description = R"(Calculates the Jaro similarity between two byte-string.)"}); + + factory.registerFunction( + FunctionDocumentation{.description = R"(Calculates the Jaro-Winkler similarity between two byte-string.)"}); } } diff --git a/tests/queries/0_stateless/02884_string_distance_function.reference b/tests/queries/0_stateless/02884_string_distance_function.reference index 0b370a43483..e8cce2017d9 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.reference +++ b/tests/queries/0_stateless/02884_string_distance_function.reference @@ -4,6 +4,8 @@ clickhouse mouse 6 clickhouse mouse 6 clickhouse mouse 0.4 clickhouse mouse 0.4 +clickhouse mouse 0 +clickhouse mouse 0 -- test aliases clickhouse mouse 10 clickhouse mouse 6 @@ -44,6 +46,20 @@ stringJaccardIndexUTF8 abc ab 0.6666666666666666 stringJaccardIndexUTF8 abc abc 1 stringJaccardIndexUTF8 abc bc 0.6666666666666666 stringJaccardIndexUTF8 clickhouse mouse 0.4 +jaroSimilarity 0 +jaroSimilarity abc 3 +jaroSimilarity abc 3 +jaroSimilarity abc ab 0.8888888888888888 +jaroSimilarity abc abc 1 +jaroSimilarity abc bc 0 +jaroSimilarity clickhouse mouse 0 +jaroWinklerSimilarity 0 +jaroWinklerSimilarity abc 3 +jaroWinklerSimilarity abc 3 +jaroWinklerSimilarity abc ab 0.9111111111111111 +jaroWinklerSimilarity abc abc 1 +jaroWinklerSimilarity abc bc 0 +jaroWinklerSimilarity clickhouse mouse 0 -- Special UTF-8 tests 0.4 0 diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index c9c8f949249..71a534bec85 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -5,6 +5,8 @@ SELECT 'clickhouse' AS s1, 'mouse' AS s2, editDistance(s1, s2); SELECT 'clickhouse' AS s1, 'mouse' AS s2, damerauLevenshteinDistance(s1, s2); SELECT 'clickhouse' AS s1, 'mouse' AS s2, stringJaccardIndex(s1, s2); SELECT 'clickhouse' AS s1, 'mouse' AS s2, stringJaccardIndexUTF8(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, jaroSimilarity(s1, s2); +SELECT 'clickhouse' AS s1, 'mouse' AS s2, jaroWinklerSimilarity(s1, s2); SELECT '-- test aliases'; SELECT 'clickhouse' AS s1, 'mouse' AS s2, mismatches(s1, s2); @@ -30,6 +32,8 @@ SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY ALL; +SELECT 'jaroSimilarity', s1, s2, jaroSimilarity(s1, s2) FROM t ORDER BY ALL; +SELECT 'jaroWinklerSimilarity', s1, s2, jaroWinklerSimilarity(s1, s2) FROM t ORDER BY ALL; SELECT '-- Special UTF-8 tests'; -- We do not perform full UTF8 validation, so sometimes it just returns some result From 5f105a56afe11b5ae599c636013758c48855f597 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 21:58:27 +0000 Subject: [PATCH 081/116] Fix style --- src/Functions/FunctionsStringDistance.cpp | 15 ++++++++------- .../check-style/aspell-ignore/en/aspell-dict.txt | 14 ++++++++++---- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 86f5a48b7b8..3bb40e1c47f 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -335,8 +335,8 @@ struct ByteDamerauLevenshteinDistanceImpl } }; -struct ByteJaroSimilarityImpl { - +struct ByteJaroSimilarityImpl +{ using ResultType = Float64; static ResultType process( @@ -376,7 +376,8 @@ struct ByteJaroSimilarityImpl { const int max_index = std::min(i + max_range + 1, s2len); for (int j = min_index; j < max_index; j++) { - if (s2_matching[j] == -1 && haystack[i] == needle[j]) { + if (s2_matching[j] == -1 && haystack[i] == needle[j]) + { s1_matching[i] = i; s2_matching[j] = j; matching_characters++; @@ -402,15 +403,15 @@ struct ByteJaroSimilarityImpl { s2i++; } double m = static_cast(matching_characters); - double jaro_similarity = 1.0 / 3.0 * ( m / static_cast(s1len) + double jaro_similarity = 1.0 / 3.0 * (m / static_cast(s1len) + m / static_cast(s2len) - + (m - transpositions) / m ); + + (m - transpositions) / m); return jaro_similarity; } }; -struct ByteJaroWinklerSimilarityImpl { - +struct ByteJaroWinklerSimilarityImpl +{ using ResultType = Float64; static ResultType process( diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d68330771e5..7c7c29d6b21 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -198,6 +198,7 @@ DELETEs DESC DIEs DOGEFI +Damerau DataGrip DataLens DataTime @@ -413,6 +414,7 @@ JSONType JSONs Jaeger Jannis +Jaro JavaHash Jemalloc Jepsen @@ -701,8 +703,6 @@ PrettySpaceMonoBlock PrettySpaceNoEscapes PrettySpaceNoEscapesMonoBlock Prewhere -TotalPrimaryKeyBytesInMemory -TotalPrimaryKeyBytesInMemoryAllocated PrivateKeyPassphraseHandler ProfileEvents Profiler @@ -924,6 +924,8 @@ Toolset TopK TotalBytesOfMergeTreeTables TotalPartsOfMergeTreeTables +TotalPrimaryKeyBytesInMemory +TotalPrimaryKeyBytesInMemoryAllocated TotalRowsOfMergeTreeTables TotalTemporaryFiles Tradeoff @@ -976,6 +978,7 @@ Werror Wether WikiStat WindowView +Winkler WithNames WithNamesAndTypes WordNet @@ -1338,6 +1341,7 @@ cutToFirstSignificantSubdomainWithWWW cutURLParameter cutWWW cyrus +damerauLevenshteinDistance datacenter datacenters datafiles @@ -1695,6 +1699,8 @@ isZeroOrNull iteratively jaccard jaccardIndex +jaroSimilarity +jaroWinklerSimilarity javaHash javaHashUTF jbod @@ -1848,14 +1854,14 @@ metrica metroHash mfedotov minMap +minSampleSizeContinuous +minSampleSizeConversion mindsdb minimalistic mininum miniselect minmap minmax -minSampleSizeContinuous -minSampleSizeConversion mins misconfiguration mispredictions From aa714a6e22106765fff13c03f9ec418eb6349355 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 08:44:50 +0000 Subject: [PATCH 082/116] Minor test adjustment --- tests/queries/0_stateless/02884_string_distance_function.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index 71a534bec85..fddbf41f0e5 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -15,6 +15,8 @@ SELECT 'clickhouse' AS s1, 'mouse' AS s2, levenshteinDistance(s1, s2); SELECT '-- Deny DoS using too large inputs'; SELECT editDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} SELECT damerauLevenshteinDistance(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} +SELECT jaroSimilarity(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} +SELECT jaroWinklerSimilarity(randomString(power(2, 17)), 'abc'); -- { serverError TOO_LARGE_STRING_SIZE} DROP TABLE IF EXISTS t; CREATE TABLE t From 21d14e488b5a475250c1e32f5d31a822d9c946a4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 16:47:12 +0000 Subject: [PATCH 083/116] Cosmetics --- contrib/CMakeLists.txt | 12 ++--- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 6 +-- contrib/qatlib-cmake/CMakeLists.txt | 46 +++++++++++++------- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 4f3c5dffb35..c5aad603fc0 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -171,7 +171,7 @@ add_contrib (s2geometry-cmake s2geometry) add_contrib (c-ares-cmake c-ares) if (OS_LINUX AND ARCH_AMD64 AND ENABLE_SSE42) - option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES}) + option (ENABLE_QPL "Enable Intel® Query Processing Library (QPL)" ${ENABLE_LIBRARIES}) elseif(ENABLE_QPL) message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 with SSE 4.2 or higher") endif() @@ -183,25 +183,25 @@ else() endif () if (OS_LINUX AND ARCH_AMD64) - option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology (Intel® QATlib)" ${ENABLE_LIBRARIES}) + option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology Library (QATlib)" ${ENABLE_LIBRARIES}) elseif(ENABLE_QATLIB) - message (${RECONFIGURE_MESSAGE_LEVEL} "QAT library is only supported on x86_64") + message (${RECONFIGURE_MESSAGE_LEVEL} "QATLib is only supported on x86_64") endif() if (ENABLE_QATLIB) option (ENABLE_QAT_USDM_DRIVER "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF) - option (ENABLE_QAT_OUT_OF_TREE_BUILD "Using out-of-tree driver, user needs customize ICP_ROOT variable" OFF) + option (ENABLE_QAT_OUT_OF_TREE_BUILD "Using out-of-tree driver, user needs to customize ICP_ROOT variable" OFF) set(ICP_ROOT "" CACHE STRING "ICP_ROOT variable to define the path of out-of-tree driver package") if (ENABLE_QAT_OUT_OF_TREE_BUILD) if (ICP_ROOT STREQUAL "") message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUT_OF_TREE_BUILD=OFF; \ - If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") + If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html") endif () else() add_contrib (qatlib-cmake qatlib) # requires: isa-l endif () add_contrib (QAT-ZSTD-Plugin-cmake QAT-ZSTD-Plugin) else() - message(STATUS "Not using QAT") + message(STATUS "Not using QATLib") endif () add_contrib (morton-nd-cmake morton-nd) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index e5a275e92e0..5a7726056e9 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -29,10 +29,7 @@ if (ENABLE_QAT_OUT_OF_TREE_BUILD) ${ZSTD_LIBRARY_DIR}) target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) - -else () - - # Intel QATZSTD in-tree build +else () # In-tree build message(STATUS "Intel QATZSTD in-tree build") set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src") set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c") @@ -84,6 +81,5 @@ else () target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTDQAT_COMPRESSION -DINTREE) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) - endif () diff --git a/contrib/qatlib-cmake/CMakeLists.txt b/contrib/qatlib-cmake/CMakeLists.txt index 3dc69ff4f25..d599775035a 100644 --- a/contrib/qatlib-cmake/CMakeLists.txt +++ b/contrib/qatlib-cmake/CMakeLists.txt @@ -1,21 +1,27 @@ # Intel® QuickAssist Technology Library (QATlib). + message(STATUS "Intel QATlib ON") set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib") set(LIBQAT_DIR "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src") set(LIBOSAL_DIR "${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src") set(OPENSSL_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl") -# Generate 3 library targets: _qatmgr, _osal, _qatlib, Output ch_contrib::qatlib by linking with these targets. -#_qatmgr -SET(LIBQATMGR_Sources ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_client.c +# Build 3 libraries: _qatmgr, _osal, _qatlib +# Produce ch_contrib::qatlib by linking these libraries. + +# _qatmgr + +SET(LIBQATMGR_sources ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_client.c ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_lib.c ${LIBQAT_DIR}/qat_direct/vfio/qat_log.c ${LIBQAT_DIR}/qat_direct/vfio/vfio_lib.c ${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_proto.c ${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_vf_msg.c ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_pf.c) -add_library(_qatmgr ${LIBQATMGR_Sources}) -target_include_directories(_qatmgr PRIVATE + +add_library(_qatmgr ${LIBQATMGR_sources}) + +target_include_directories(_qatmgr PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/vfio ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include ${LIBQAT_ROOT_DIR}/quickassist/include @@ -25,11 +31,13 @@ target_include_directories(_qatmgr PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/include ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) + target_compile_definitions(_qatmgr PRIVATE -DUSER_SPACE) target_compile_options(_qatmgr PRIVATE -Wno-error=int-conversion) -#_osal -SET(LIBOSAL_Sources +# _osal + +SET(LIBOSAL_sources ${LIBOSAL_DIR}/linux/user_space/OsalSemaphore.c ${LIBOSAL_DIR}/linux/user_space/OsalThread.c ${LIBOSAL_DIR}/linux/user_space/OsalMutex.c @@ -38,7 +46,9 @@ SET(LIBOSAL_Sources ${LIBOSAL_DIR}/linux/user_space/OsalServices.c ${LIBOSAL_DIR}/linux/user_space/OsalUsrKrnProxy.c ${LIBOSAL_DIR}/linux/user_space/OsalCryptoInterface.c) -add_library(_osal ${LIBOSAL_Sources}) + +add_library(_osal ${LIBOSAL_sources}) + target_include_directories(_osal PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include @@ -46,10 +56,11 @@ target_include_directories(_osal PRIVATE ${OPENSSL_DIR}/include ${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include ${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include) + target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL) -#_qatlib -SET(LIBQAT_Sources +# _qatlib +SET(LIBQAT_sources ${LIBQAT_DIR}/common/compression/dc_buffers.c ${LIBQAT_DIR}/common/compression/dc_chain.c ${LIBQAT_DIR}/common/compression/dc_datapath.c @@ -153,7 +164,8 @@ SET(LIBQAT_Sources ${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_proxy.c ${LIBQAT_DIR}/common/compression/dc_crc_base.c) -add_library(_qatlib ${LIBQAT_Sources}) +add_library(_qatlib ${LIBQAT_sources}) + target_include_directories(_qatlib PRIVATE ${CMAKE_SYSROOT}/usr/include ${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include @@ -175,23 +187,27 @@ target_include_directories(_qatlib PRIVATE ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space ${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include) + target_link_libraries(_qatlib PRIVATE _qatmgr _osal OpenSSL::SSL ch_contrib::isal) target_compile_definitions(_qatlib PRIVATE -DUSER_SPACE -DLAC_BYTE_ORDER=__LITTLE_ENDIAN -DOSAL_ENSURE_ON) target_link_options(_qatlib PRIVATE -pie -z relro -z now -z noexecstack) target_compile_options(_qatlib PRIVATE -march=native) add_library (ch_contrib::qatlib ALIAS _qatlib) -#_usdm +# _usdm + set(LIBUSDM_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib/quickassist/utilities/libusdm_drv") -set(LIBUSDM_Sources +set(LIBUSDM_sources ${LIBUSDM_DIR}/user_space/vfio/qae_mem_utils_vfio.c ${LIBUSDM_DIR}/user_space/qae_mem_utils_common.c ${LIBUSDM_DIR}/user_space/vfio/qae_mem_hugepage_utils_vfio.c) -add_library(_usdm ${LIBUSDM_Sources}) + +add_library(_usdm ${LIBUSDM_sources}) target_include_directories(_usdm PRIVATE ${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include ${LIBUSDM_DIR} ${LIBUSDM_DIR}/include ${LIBUSDM_DIR}/user_space) -add_library (ch_contrib::usdm ALIAS _usdm) \ No newline at end of file + +add_library (ch_contrib::usdm ALIAS _usdm) From 0f51e32bb59073d042de87d50d56761efd8de7e5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 17:47:31 +0000 Subject: [PATCH 084/116] Rename from ZSTDQPL to ZSTD_QPL --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 2 +- docs/en/operations/settings/settings.md | 24 +++++++- .../sql-reference/statements/create/table.md | 13 +++-- src/Compression/CompressionCodecZSTD.cpp | 12 +++- src/Compression/CompressionCodecZSTDQAT.cpp | 57 ++++++++++--------- src/Compression/CompressionFactory.cpp | 4 +- .../CompressionFactoryAdditions.cpp | 2 +- src/Compression/ICompressionCodec.h | 4 +- src/Core/Settings.h | 2 +- ..._test_zstd_qat_codec_compression.reference | 6 ++ ...00804_test_zstd_qat_codec_compression.sql} | 30 +++++----- .../aspell-ignore/en/aspell-dict.txt | 2 +- 12 files changed, 98 insertions(+), 60 deletions(-) create mode 100644 tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference rename tests/queries/0_stateless/{00804_test_zstdqat_codec_compression.sql => 00804_test_zstd_qat_codec_compression.sql} (65%) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 5a7726056e9..45a956dcb55 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -78,7 +78,7 @@ else () # In-tree build ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR} ${LIBQAT_HEADER_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTDQAT_COMPRESSION -DINTREE) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC -DINTREE) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) endif () diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index d4ee8106320..3b44d537972 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4773,6 +4773,28 @@ Type: Int64 Default: 0 +## enable_deflate_qpl_codec {#enable_deflate_qpl_codec} + +If turned on, the DEFLATE_QPL codec may be used to compress columns. + +Possible values: + +- 0 - Disabled +- 1 - Enabled + +Type: Bool + +## enable_zstd_qat_codec {#enable_zstd_qat_codec} + +If turned on, the ZSTD_QAT codec may be used to compress columns. + +Possible values: + +- 0 - Disabled +- 1 - Enabled + +Type: Bool + ## rewrite_count_distinct_if_with_count_distinct_implementation Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#count_distinct_implementation) setting. @@ -5157,4 +5179,4 @@ The value 0 means that you can delete all tables without any restrictions. :::note This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop) -::: \ No newline at end of file +::: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 9b4d11fda46..0258c64e422 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -372,22 +372,23 @@ ClickHouse supports general purpose codecs and specialized codecs. #### ZSTD -`ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: \[1, 22\]. Default value: 1. +`ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: \[1, 22\]. Default level: 1. High compression levels are useful for asymmetric scenarios, like compress once, decompress repeatedly. Higher levels mean better compression and higher CPU usage. -#### ZSTDQAT +#### ZSTD_QAT -`ZSTDQAT[(level)]` — ZSTD QAT (hardware-accelerated compression) implemented by [QAT-ZSTD-Plugin](https://github.com/intel/QAT-ZSTD-Plugin) with configurable level. Default level: 1. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[6, 12\]. +`ZSTD_QAT[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable level, implemented by [Intel® QATlib](https://github.com/intel/qatlib) and [Intel® QAT ZSTD Plugin](https://github.com/intel/QAT-ZSTD-Plugin). Possible levels: \[1, 12\]. Default level: 1. Recommended level range: \[6, 12\]. Some limitations apply: -- ZSTDQAT is disabled by default and can only be used after setting configuration parameter `enable_zstd_qat_codec = 1`. -- ZSTDQAT tries to use an Intel® QAT offloading device ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)). If no such device was found, it will fallback to ZSTD compression in software. +- ZSTD_QAT is disabled by default and can only be used after enabling configuration setting [enable_zstd_qat_codec](../../../operations/settings/settings.md#enable_zstd_qat_codec). +- For compression, ZSTD_QAT tries to use an Intel® QAT offloading device ([QuickAssist Technology](https://www.intel.com/content/www/us/en/developer/topic-technology/open/quick-assist-technology/overview.html)). If no such device was found, it will fallback to ZSTD compression in software. +- Decompression is always performed in software. #### DEFLATE_QPL `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: -- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_deflate_qpl_codec = 1`. +- DEFLATE_QPL is disabled by default and can only be used after enabling configuration setting [enable_deflate_qpl_codec](../../../operations/settings/settings.md#enable_deflate_qpl_codec). - DEFLATE_QPL requires a ClickHouse build compiled with SSE 4.2 instructions (by default, this is the case). Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. - DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled. diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 5b7a8809bbc..7aecb652efc 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -51,7 +51,7 @@ UInt32 CompressionCodecZSTD::doCompressData(const char * source, UInt32 source_s ZSTD_freeCCtx(cctx); if (ZSTD_isError(compressed_size)) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", ZSTD_getErrorName(compressed_size)); return static_cast(compressed_size); } @@ -65,13 +65,19 @@ void CompressionCodecZSTD::doDecompressData(const char * source, UInt32 source_s throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress ZSTD-encoded data: {}", std::string(ZSTD_getErrorName(res))); } -CompressionCodecZSTD::CompressionCodecZSTD(int level_, int window_log_) : level(level_), enable_long_range(true), window_log(window_log_) +CompressionCodecZSTD::CompressionCodecZSTD(int level_, int window_log_) + : level(level_) + , enable_long_range(true) + , window_log(window_log_) { setCodecDescription( "ZSTD", {std::make_shared(static_cast(level)), std::make_shared(static_cast(window_log))}); } -CompressionCodecZSTD::CompressionCodecZSTD(int level_) : level(level_), enable_long_range(false), window_log(0) +CompressionCodecZSTD::CompressionCodecZSTD(int level_) + : level(level_) + , enable_long_range(false) + , window_log(0) { setCodecDescription("ZSTD", {std::make_shared(static_cast(level))}); } diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 8760f60cd69..4c9a0549a61 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,15 +1,12 @@ -#ifdef ENABLE_ZSTDQAT_COMPRESSION -#include -#include -#include -#include -#include -#include -#include -#include -#include +#ifdef ENABLE_ZSTD_QAT_CODEC #include +#include +#include +#include +#include + #include +#include namespace DB { @@ -20,18 +17,18 @@ namespace ErrorCodes extern const int ILLEGAL_CODEC_PARAMETER; } +/// Hardware-accelerated ZSTD. Supports only compression so far. class CompressionCodecZSTDQAT : public CompressionCodecZSTD { public: - /// QAT Hardware only supports compression levels L1 to L12 static constexpr auto ZSTDQAT_SUPPORTED_MIN_LEVEL = 1; static constexpr auto ZSTDQAT_SUPPORTED_MAX_LEVEL = 12; + explicit CompressionCodecZSTDQAT(int level_); ~CompressionCodecZSTDQAT() override; protected: - bool isZSTDQAT() const override { return true; } - /// TODO: So far, QAT hardware only support compression. For next generation in future, it will support decompression as well. + bool isZstdQat() const override { return true; } UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; private: @@ -46,28 +43,29 @@ UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 sourc size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); if (ZSTD_isError(compressed_size)) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD_QAT codec: {}", ZSTD_getErrorName(compressed_size)); return static_cast(compressed_size); } void registerCodecZSTDQAT(CompressionCodecFactory & factory) { - factory.registerCompressionCodec("ZSTDQAT", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + factory.registerCompressionCodec("ZSTD_QAT", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr { int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; if (arguments && !arguments->children.empty()) { if (arguments->children.size() > 1) - throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "ZSTDQAT codec must have 1 parameter, given {}", arguments->children.size()); + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "ZSTD_QAT codec must have 1 parameter, given {}", arguments->children.size()); const auto children = arguments->children; const auto * literal = children[0]->as(); if (!literal) - throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTDQAT codec argument must be integer"); + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTD_QAT codec argument must be integer"); level = static_cast(literal->value.safeGet()); - if (level > CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL || level < CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL) + if (level < CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL || level > CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL ) + /// that's a hardware limitation throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTDQAT codec doesn't support level more than {} and lower than {} , given {}", CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL, CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL, level); @@ -78,39 +76,42 @@ void registerCodecZSTDQAT(CompressionCodecFactory & factory) } CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) - : CompressionCodecZSTD(level_), level(level_), log(&Poco::Logger::get("CompressionCodecZSTDQAT")) + : CompressionCodecZSTD(level_) + , level(level_) + , log(&Poco::Logger::get("CompressionCodecZSTDQAT")) { - setCodecDescription("ZSTDQAT", {std::make_shared(static_cast(level))}); + setCodecDescription("ZSTD_QAT", {std::make_shared(static_cast(level))}); + cctx = ZSTD_createCCtx(); - /// Start QAT device, start QAT device at any time before compression job started + int res = QZSTD_startQatDevice(); - LOG_WARNING(log, "Initialization of hardware-assisted(QAT) ZSTD codec result: {} ", static_cast(res)); - /// Create sequence producer state for QAT sequence producer + LOG_DEBUG(log, "Initialization of ZSTD_QAT codec, status: {} ", res); + sequenceProducerState = QZSTD_createSeqProdState(); - /// register qatSequenceProducer + ZSTD_registerSequenceProducer( cctx, sequenceProducerState, qatSequenceProducer ); - /// Enable sequence producer fallback + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); } CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() { - /// Free sequence producer state if (sequenceProducerState != nullptr) { QZSTD_freeSeqProdState(sequenceProducerState); sequenceProducerState = nullptr; } + if (cctx != nullptr) { - auto status = ZSTD_freeCCtx(cctx); + size_t status = ZSTD_freeCCtx(cctx); if (status != 0) - LOG_WARNING(log, "ZSTD_freeCCtx failed with status: {} ", static_cast(status)); + LOG_WARNING(log, "ZSTD_freeCCtx failed with status: {} ", status); cctx = nullptr; } } diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 6cc34c6aa91..f4413401667 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -167,7 +167,7 @@ void registerCodecNone(CompressionCodecFactory & factory); void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); -#ifdef ENABLE_ZSTDQAT_COMPRESSION +#ifdef ENABLE_ZSTD_QAT_CODEC void registerCodecZSTDQAT(CompressionCodecFactory & factory); #endif void registerCodecMultiple(CompressionCodecFactory & factory); @@ -192,7 +192,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecNone(*this); registerCodecLZ4(*this); registerCodecZSTD(*this); -#ifdef ENABLE_ZSTDQAT_COMPRESSION +#ifdef ENABLE_ZSTD_QAT_CODEC registerCodecZSTDQAT(*this); #endif registerCodecLZ4HC(*this); diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index a3612ccb4b2..f4d993f628e 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -165,7 +165,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'enable_deflate_qpl_codec' setting.", codec_family_name); - if (!enable_zstd_qat_codec && result_codec->isZSTDQAT()) + if (!enable_zstd_qat_codec && result_codec->isZstdQat()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec {} is disabled by default." " You can enable it with the 'enable_zstd_qat_codec' setting.", diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index ebf8591d222..18ff543d908 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -121,8 +121,8 @@ public: /// Is this the DEFLATE_QPL codec? virtual bool isDeflateQpl() const { return false; } - /// Is this the ZSTDQAT codec? - virtual bool isZSTDQAT() const { return false; } + /// Is this the ZSTD_QAT codec? + virtual bool isZstdQat() const { return false; } /// If it does nothing. virtual bool isNone() const { return false; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 85461d1cb41..5bcecbba7f8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -352,7 +352,7 @@ class IColumn; M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(Bool, enable_deflate_qpl_codec, false, "Enable/disable the DEFLATE_QPL codec.", 0) \ - M(Bool, enable_zstd_qat_codec, false, "Enable/disable the ZSTDQAT codec.", 0) \ + M(Bool, enable_zstd_qat_codec, false, "Enable/disable the ZSTD_QAT codec.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference new file mode 100644 index 00000000000..31a4360469f --- /dev/null +++ b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference @@ -0,0 +1,6 @@ +CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(ZSTD_QAT(1)),\n `data` String CODEC(ZSTD_QAT(1)),\n `ddd` Date CODEC(ZSTD_QAT(1)),\n `ddd32` Date32 CODEC(ZSTD_QAT(1)),\n `somenum` Float64 CODEC(ZSTD_QAT(1)),\n `somestr` FixedString(3) CODEC(ZSTD_QAT(1)),\n `othernum` Int64 CODEC(ZSTD_QAT(1)),\n `somearray` Array(UInt8) CODEC(ZSTD_QAT(1)),\n `somemap` Map(String, UInt32) CODEC(ZSTD_QAT(1)),\n `sometuple` Tuple(UInt16, UInt64) CODEC(ZSTD_QAT(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2) +2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4) +3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6) +2 +10001 diff --git a/tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql similarity index 65% rename from tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql rename to tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql index e79b4fb06f5..95943dd222c 100644 --- a/tests/queries/0_stateless/00804_test_zstdqat_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql @@ -1,8 +1,6 @@ --Tags: no-fasttest, no-cpu-aarch64, no-cpu-s390x --- no-fasttest because ZSTDQAT isn't available in fasttest --- no-cpu-aarch64 and no-cpu-s390x because ZSTDQAT is x86-only - --- A bunch of random DDLs to test the ZSTDQAT codec. +-- no-fasttest because ZSTD_QAT isn't available in fasttest +-- no-cpu-aarch64 and no-cpu-s390x because ZSTD_QAT is x86-only SET enable_zstd_qat_codec = 1; @@ -12,17 +10,21 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS compression_codec; +-- negative test +CREATE TABLE compression_codec(id UInt64 CODEC(ZSTD_QAT(0))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_CODEC_PARAMETER } +CREATE TABLE compression_codec(id UInt64 CODEC(ZSTD_QAT(13))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_CODEC_PARAMETER } + CREATE TABLE compression_codec( - id UInt64 CODEC(ZSTDQAT), - data String CODEC(ZSTDQAT), - ddd Date CODEC(ZSTDQAT), - ddd32 Date32 CODEC(ZSTDQAT), - somenum Float64 CODEC(ZSTDQAT), - somestr FixedString(3) CODEC(ZSTDQAT), - othernum Int64 CODEC(ZSTDQAT), - somearray Array(UInt8) CODEC(ZSTDQAT), - somemap Map(String, UInt32) CODEC(ZSTDQAT), - sometuple Tuple(UInt16, UInt64) CODEC(ZSTDQAT), + id UInt64 CODEC(ZSTD_QAT), + data String CODEC(ZSTD_QAT), + ddd Date CODEC(ZSTD_QAT), + ddd32 Date32 CODEC(ZSTD_QAT), + somenum Float64 CODEC(ZSTD_QAT), + somestr FixedString(3) CODEC(ZSTD_QAT), + othernum Int64 CODEC(ZSTD_QAT), + somearray Array(UInt8) CODEC(ZSTD_QAT), + somemap Map(String, UInt32) CODEC(ZSTD_QAT), + sometuple Tuple(UInt16, UInt64) CODEC(ZSTD_QAT), ) ENGINE = MergeTree() ORDER BY tuple(); SHOW CREATE TABLE compression_codec; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 1a6f148365a..c50d7092b3e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1001,7 +1001,7 @@ ZooKeeperRequest ZooKeeperSession ZooKeeperWatch ZooKeepers -ZSTDQAT +ZSTD_QAT aarch accurateCast accurateCastOrDefault From 8a9d4cbf8a726c1cb62a72dcd9c09ac7cfa852a8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 17:52:13 +0000 Subject: [PATCH 085/116] Add method byte --- src/Compression/CompressionCodecZSTDQAT.cpp | 3 ++- src/Compression/CompressionInfo.h | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 4c9a0549a61..47e16cc698a 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -50,7 +50,8 @@ UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 sourc void registerCodecZSTDQAT(CompressionCodecFactory & factory) { - factory.registerCompressionCodec("ZSTD_QAT", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + UInt8 method_code = static_cast(CompressionMethodByte::ZSTD_QPL); + factory.registerCompressionCodec("ZSTD_QAT", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr { int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; if (arguments && !arguments->children.empty()) diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index 1b4025fed1d..ee4b3e38653 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -48,6 +48,7 @@ enum class CompressionMethodByte : uint8_t FPC = 0x98, DeflateQpl = 0x99, GCD = 0x9a, + ZSTD_QPL = 0x9b, }; } From 14cda082d20d8e04de9d6abfc8b8d9291152d09b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 18:08:52 +0000 Subject: [PATCH 086/116] Fix style --- src/Compression/CompressionCodecZSTDQAT.cpp | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 47e16cc698a..e3a6714f191 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -65,7 +65,7 @@ void registerCodecZSTDQAT(CompressionCodecFactory & factory) throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTD_QAT codec argument must be integer"); level = static_cast(literal->value.safeGet()); - if (level < CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL || level > CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL ) + if (level < CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MIN_LEVEL || level > CompressionCodecZSTDQAT::ZSTDQAT_SUPPORTED_MAX_LEVEL) /// that's a hardware limitation throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTDQAT codec doesn't support level more than {} and lower than {} , given {}", diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c50d7092b3e..e15de2dbed0 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -701,8 +701,6 @@ PrettySpaceMonoBlock PrettySpaceNoEscapes PrettySpaceNoEscapesMonoBlock Prewhere -TotalPrimaryKeyBytesInMemory -TotalPrimaryKeyBytesInMemoryAllocated PrivateKeyPassphraseHandler ProfileEvents Profiler @@ -716,6 +714,7 @@ ProtobufSingle ProxySQL PyArrow PyCharm +QATlib QEMU QTCreator Quantile @@ -925,6 +924,8 @@ Toolset TopK TotalBytesOfMergeTreeTables TotalPartsOfMergeTreeTables +TotalPrimaryKeyBytesInMemory +TotalPrimaryKeyBytesInMemoryAllocated TotalRowsOfMergeTreeTables TotalTemporaryFiles Tradeoff @@ -993,6 +994,7 @@ YYYYMMDDToDate YYYYMMDDhhmmssToDateTime Yandex Yasm +ZSTDQAT Zabbix Zipkin ZooKeeper @@ -1001,7 +1003,6 @@ ZooKeeperRequest ZooKeeperSession ZooKeeperWatch ZooKeepers -ZSTD_QAT aarch accurateCast accurateCastOrDefault @@ -1850,14 +1851,14 @@ metrica metroHash mfedotov minMap +minSampleSizeContinuous +minSampleSizeConversion mindsdb minimalistic mininum miniselect minmap minmax -minSampleSizeContinuous -minSampleSizeConversion mins misconfiguration mispredictions From 14a869e5189e52833e14fddc883c1e0dfd16ce14 Mon Sep 17 00:00:00 2001 From: Eyal Halpern Shalev Date: Sun, 7 Jan 2024 21:10:57 +0200 Subject: [PATCH 087/116] Support CLICKHOUSE_PASSWORD_FILE environment variable in Dockerfile Implement #58575 --- docker/server/entrypoint.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index d94ffb893e1..b9c7ea34a36 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -41,6 +41,10 @@ readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLIC readarray -t DISKS_METADATA_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.metadata_path' || true) CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}" +CLICKHOUSE_PASSWORD_FILE="${CLICKHOUSE_PASSWORD_FILE:-}" +if [[ -n "${CLICKHOUSE_PASSWORD_FILE}" && -f "${CLICKHOUSE_PASSWORD_FILE}" ]]; then + CLICKHOUSE_PASSWORD="$(cat "${CLICKHOUSE_PASSWORD_FILE}")" +fi CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}" CLICKHOUSE_DB="${CLICKHOUSE_DB:-}" CLICKHOUSE_ACCESS_MANAGEMENT="${CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT:-0}" From 6f626d829433d9001fa9f5f900ef394c75eb930c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 7 Jan 2024 23:00:26 +0100 Subject: [PATCH 088/116] fix auth_settings.hasUpdates function --- src/IO/S3Common.cpp | 2 +- src/Storages/StorageS3.cpp | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index f2ead22127c..5039059f522 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -142,7 +142,7 @@ bool AuthSettings::hasUpdates(const AuthSettings & other) const { AuthSettings copy = *this; copy.updateFrom(other); - return copy != other; + return *this != copy; } void AuthSettings::updateFrom(const AuthSettings & from) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 0909539033d..086cb0acbae 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1429,7 +1429,7 @@ bool StorageS3::Configuration::update(ContextPtr context) request_settings = s3_settings.request_settings; request_settings.updateFromSettings(context->getSettings()); - if (client && (static_configuration || !s3_settings.auth_settings.hasUpdates(auth_settings))) + if (client && (static_configuration || !auth_settings.hasUpdates(s3_settings.auth_settings))) return false; auth_settings.updateFrom(s3_settings.auth_settings); @@ -1652,10 +1652,11 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context if (engine_args_to_idx.contains("session_token")) configuration.auth_settings.session_token = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["session_token"]], "session_token"); - configuration.auth_settings.no_sign_request = no_sign_request; + if (no_sign_request) + configuration.auth_settings.no_sign_request = no_sign_request; } - configuration.static_configuration = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request; + configuration.static_configuration = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request.has_value(); configuration.keys = {configuration.url.key}; From 29173a602e45391cf57bd152cd638ffe4719d2b7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Jan 2024 02:39:19 +0000 Subject: [PATCH 089/116] Automatic style fix --- .../integration/test_storage_s3_queue/test.py | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 6fee339c29b..7d40060fec6 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -21,9 +21,7 @@ def prepare_public_s3_bucket(started_cluster): client.make_bucket(bucket_name) - client.set_bucket_policy( - bucket_name, json.dumps(policy) - ) + client.set_bucket_policy(bucket_name, json.dumps(policy)) def get_policy_with_public_access(bucket_name): return { @@ -72,9 +70,7 @@ def s3_queue_setup_teardown(started_cluster): instance_2.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") minio = started_cluster.minio_client - objects = list( - minio.list_objects(started_cluster.minio_bucket, recursive=True) - ) + objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) for obj in objects: minio.remove_object(started_cluster.minio_bucket, obj.object_name) yield # run test @@ -124,7 +120,13 @@ def run_query(instance, query, stdin=None, settings=None): def generate_random_files( - started_cluster, files_path, count, column_num=3, row_num=10, start_ind=0, bucket=None + started_cluster, + files_path, + count, + column_num=3, + row_num=10, + start_ind=0, + bucket=None, ): files = [ (f"{files_path}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) @@ -149,9 +151,7 @@ def generate_random_files( def put_s3_file_content(started_cluster, filename, data, bucket=None): bucket = started_cluster.minio_bucket if bucket is None else bucket buf = io.BytesIO(data) - started_cluster.minio_client.put_object( - bucket, filename, buf, len(data) - ) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) def create_table( @@ -903,9 +903,10 @@ def test_s3_client_reused(started_cluster): row_num = 10 def get_created_s3_clients_count(): - value = node.query(f"SELECT value FROM system.events WHERE event='S3Clients'").strip() - return int(value) if value != '' else 0 - + value = node.query( + f"SELECT value FROM system.events WHERE event='S3Clients'" + ).strip() + return int(value) if value != "" else 0 def wait_all_processed(files_num): expected_count = files_num * row_num @@ -915,7 +916,9 @@ def test_s3_client_reused(started_cluster): if count == expected_count: break time.sleep(1) - assert int(node.query(f"SELECT count() FROM {dst_table_name}")) == expected_count + assert ( + int(node.query(f"SELECT count() FROM {dst_table_name}")) == expected_count + ) prepare_public_s3_bucket(started_cluster) @@ -944,10 +947,15 @@ def test_s3_client_reused(started_cluster): s3_clients_before = get_created_s3_clients_count() generate_random_files( - started_cluster, files_path, count=1, start_ind=i, row_num=row_num, bucket=started_cluster.minio_public_bucket + started_cluster, + files_path, + count=1, + start_ind=i, + row_num=row_num, + bucket=started_cluster.minio_public_bucket, ) - wait_all_processed(i+1) + wait_all_processed(i + 1) s3_clients_after = get_created_s3_clients_count() From a293ade11bb6987f3a3a0240d0f16caa2e32bfbb Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 8 Jan 2024 14:43:29 +0800 Subject: [PATCH 090/116] modify code --- src/Functions/multiIf.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 07e9b440edd..d0f5a1ce439 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -398,8 +398,8 @@ private: PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); - int first_non_nullable_source = -1; - for (int i = 0; i < static_cast(instructions.size()); ++i) + ColumnPtr shared_null_map_col = nullptr; + for (size_t i = 0; i < instructions.size(); ++i) { if (instructions[i].source->isNullable()) { @@ -416,11 +416,11 @@ private: } else { - if (first_non_nullable_source < 0) + if (!shared_null_map_col) { - first_non_nullable_source = i; + shared_null_map_col = ColumnUInt8::create(rows, 0); } - null_map_cols[i] = (i == first_non_nullable_source ? ColumnUInt8::create(rows, 0)->getData().data() : null_map_cols[first_non_nullable_source]); + null_map_cols[i] = assert_cast(*shared_null_map_col).getData().data(); data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } From fd809956f0f62b721eafc844f6cb65338badede4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 8 Jan 2024 08:14:09 +0000 Subject: [PATCH 091/116] fix style check Signed-off-by: Duc Canh Le --- src/Core/Settings.h | 2 +- .../queries/0_stateless/02961_output_format_compress_params.sql | 0 2 files changed, 1 insertion(+), 1 deletion(-) mode change 100755 => 100644 tests/queries/0_stateless/02961_output_format_compress_params.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 254f7338a30..79a1c3a968b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -204,7 +204,7 @@ class IColumn; M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \ - M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table fuction `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`.", 0) \ + M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table function `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`.", 0) \ M(UInt64, output_format_compression_zstd_window_log, 0, "To be used when output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression.", 0) \ \ M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.sql b/tests/queries/0_stateless/02961_output_format_compress_params.sql old mode 100755 new mode 100644 From 7e195777648032b6acce8d5ae454e3245d61e20f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jan 2024 09:22:40 +0000 Subject: [PATCH 092/116] Minor fixups --- src/Compression/CompressionCodecZSTDQAT.cpp | 28 +++++++-------------- 1 file changed, 9 insertions(+), 19 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index f7ff783bf9c..0d06de6fb05 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -33,8 +33,8 @@ protected: private: const int level; - ZSTD_CCtx * cctx; - void * sequenceProducerState; + ZSTD_CCtx * cctx = nullptr; + void * sequenceProducerState = nullptr; Poco::Logger * log; }; @@ -85,21 +85,16 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) cctx = ZSTD_createCCtx(); - int res = QZSTD_startQatDevice(); - - if(res == QZSTD_OK) + const int res = QZSTD_startQatDevice(); + if (res == QZSTD_OK) { sequenceProducerState = QZSTD_createSeqProdState(); - ZSTD_registerSequenceProducer( - cctx, - sequenceProducerState, - qatSequenceProducer - ); + ZSTD_registerSequenceProducer(cctx, sequenceProducerState, qatSequenceProducer); ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - LOG_DEBUG(log, "Hardware-assisted ZSTD_QAT codec is ready!"); + LOG_DEBUG(log, "Initialization of hardware-assissted ZSTD_QAT codec successful"); } else - LOG_DEBUG(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, status: {} - please refer to QZSTD_Status_e in ./contrib/QAT-ZSTD-Plugin/src/qatseqprod.h", res); + LOG_WARNING(log, "Initialization of hardware-assisted ZSTD_QAT codec failed. Status: {} (please refer to QZSTD_Status_e in ./contrib/QAT-ZSTD-Plugin/src/qatseqprod.h)", res); ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); } @@ -112,13 +107,8 @@ CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() sequenceProducerState = nullptr; } - if (cctx != nullptr) - { - size_t status = ZSTD_freeCCtx(cctx); - if (status != 0) - LOG_WARNING(log, "ZSTD_freeCCtx failed with status: {} ", status); - cctx = nullptr; - } + ZSTD_freeCCtx(cctx); + cctx = nullptr; } } From 0f2588334f8969caae6c966535cb45625c174730 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 10:20:39 +0000 Subject: [PATCH 093/116] Disable max_joined_block_rows in ConcurrentHashJoin --- src/Interpreters/ConcurrentHashJoin.cpp | 3 +++ src/Interpreters/HashJoin.cpp | 4 ++-- src/Interpreters/HashJoin.h | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 8e73bc8b484..96be70c5527 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -46,6 +46,9 @@ ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr(); inner_hash_join->data = std::make_unique(table_join_, right_sample_block, any_take_last_row_, 0, fmt::format("concurrent{}", i)); + /// Non zero `max_joined_block_rows` allows to process block partially and return not processed part. + /// TODO: It's not handled properly in ConcurrentHashJoin case, so we set it to 0 to disable this feature. + inner_hash_join->data->setMaxJoinedBlockRows(0); hash_joins.emplace_back(std::move(inner_hash_join)); } } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a84e1ec2175..468d23ed744 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -243,6 +243,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , asof_inequality(table_join->getAsofInequality()) , data(std::make_shared()) , right_sample_block(right_sample_block_) + , max_joined_block_rows(table_join->maxJoinedBlockRows()) , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , log(&Poco::Logger::get("HashJoin")) { @@ -1690,7 +1691,7 @@ Block HashJoin::joinBlockImpl( bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = join_features.need_filter || has_required_right_keys; - added_columns.max_joined_block_rows = table_join->maxJoinedBlockRows(); + added_columns.max_joined_block_rows = max_joined_block_rows; if (!added_columns.max_joined_block_rows) added_columns.max_joined_block_rows = std::numeric_limits::max(); else @@ -1771,7 +1772,6 @@ Block HashJoin::joinBlockImpl( void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const { - size_t max_joined_block_rows = table_join->maxJoinedBlockRows(); size_t start_left_row = 0; size_t start_right_block = 0; if (not_processed) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 284cf5d0e7f..2be58b5fd2d 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -396,6 +396,8 @@ public: void shrinkStoredBlocksToFit(size_t & total_bytes_in_join); + void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } + private: template friend class NotJoinedHash; @@ -433,6 +435,9 @@ private: /// Left table column names that are sources for required_right_keys columns std::vector required_right_keys_sources; + /// Maximum number of rows in result block. If it is 0, then no limits. + size_t max_joined_block_rows = 0; + /// When tracked memory consumption is more than a threshold, we will shrink to fit stored blocks. bool shrink_blocks = false; Int64 memory_usage_before_adding_blocks = 0; From 8dad3f1b22a5f76893f01d0a2cd3dc6d5c6b2427 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 10:58:26 +0000 Subject: [PATCH 094/116] Add test 02962_max_joined_block_rows --- .../02962_max_joined_block_rows.reference | 32 ++++++++++++++++ .../02962_max_joined_block_rows.sql | 38 +++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 tests/queries/0_stateless/02962_max_joined_block_rows.reference create mode 100644 tests/queries/0_stateless/02962_max_joined_block_rows.sql diff --git a/tests/queries/0_stateless/02962_max_joined_block_rows.reference b/tests/queries/0_stateless/02962_max_joined_block_rows.reference new file mode 100644 index 00000000000..8bc1bad225b --- /dev/null +++ b/tests/queries/0_stateless/02962_max_joined_block_rows.reference @@ -0,0 +1,32 @@ +1 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +-- +1 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +-- +1 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 diff --git a/tests/queries/0_stateless/02962_max_joined_block_rows.sql b/tests/queries/0_stateless/02962_max_joined_block_rows.sql new file mode 100644 index 00000000000..c31ab5e1132 --- /dev/null +++ b/tests/queries/0_stateless/02962_max_joined_block_rows.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE table t1 (a UInt64, b UInt64) ENGINE = Memory; +INSERT INTO t1 SELECT number % 2, number FROM numbers(10); + +CREATE table t2 (a UInt64) ENGINE = Memory; + +INSERT INTO t2 SELECT number % 2 FROM numbers(10); + +-- block size is always multiple of 5 because we have 5 rows for each key in right table +-- we do not split rows corresponding to the same key + +SELECT max(bs) <= 5, b FROM ( + SELECT blockSize() as bs, * FROM t1 JOIN t2 ON t1.a = t2.a +) GROUP BY b +ORDER BY b +SETTINGS max_joined_block_size_rows = 5; + +SELECT '--'; + +SELECT max(bs) <= 10, b FROM ( + SELECT blockSize() as bs, * FROM t1 JOIN t2 ON t1.a = t2.a +) GROUP BY b +ORDER BY b +SETTINGS max_joined_block_size_rows = 10; + +SELECT '--'; + +-- parallel_hash doen't support max_joined_block_size_rows + +SET join_algorithm = 'parallel_hash'; + +SELECT max(bs) > 10, b FROM ( + SELECT blockSize() as bs, * FROM t1 JOIN t2 ON t1.a = t2.a +) GROUP BY b +ORDER BY b +SETTINGS max_joined_block_size_rows = 10; From 631e16d8c34ac3e88d215fb8e188a666faafa7c1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 11:02:17 +0000 Subject: [PATCH 095/116] Fix off-by-one error for max_joined_block_rows --- src/Interpreters/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 468d23ed744..b05b10ff25e 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1402,7 +1402,7 @@ NO_INLINE size_t joinRightColumns( { if constexpr (join_features.need_replication) { - if (unlikely(current_offset > max_joined_block_rows)) + if (unlikely(current_offset >= max_joined_block_rows)) { added_columns.offsets_to_replicate->resize_assume_reserved(i); added_columns.filter.resize_assume_reserved(i); From f3c77c046ad982a987d1beec752a030fddd657b7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jan 2024 11:58:32 +0000 Subject: [PATCH 096/116] makeDateTime64(): Allow non-const fraction argument --- src/Functions/makeDate.cpp | 2 +- tests/queries/0_stateless/02245_make_datetime64.reference | 1 + tests/queries/0_stateless/02245_make_datetime64.sql | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 1381e9f2828..987cf4eb1a9 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -434,7 +434,7 @@ public: }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isNumber, isColumnConst, "const Number"}, + {optional_argument_names[0], &isNumber, nullptr, "const Number"}, {optional_argument_names[1], &isNumber, isColumnConst, "const Number"}, {optional_argument_names[2], &isString, isColumnConst, "const String"} }; diff --git a/tests/queries/0_stateless/02245_make_datetime64.reference b/tests/queries/0_stateless/02245_make_datetime64.reference index 0ac672ae54d..1c7d31788e3 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.reference +++ b/tests/queries/0_stateless/02245_make_datetime64.reference @@ -67,3 +67,4 @@ DateTime64(7, \'UTC\') 1900-01-01 00:00:00.000 1900-01-01 00:00:00.000 1900-01-01 00:00:00.000 +2024-01-08 11:12:13.014 diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index 62784cb9b75..71629ad8dff 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -82,6 +82,9 @@ select makeDateTime64(1991, 8, 24, 65537, 4, 0); select makeDateTime64(1991, 8, 24, 21, 65537, 0); select makeDateTime64(1991, 8, 24, 21, 4, 65537); +-- bug 58590 +select makeDateTime64(2024, 1, 8, 11, 12, 13, materialize(14)); + select makeDateTime64(year, 1, 1, 1, 0, 0, 0, precision, timezone) from ( select 1984 as year, 5 as precision, 'UTC' as timezone union all From 621d17b16f859f91df1adc015404e1d4ac71a592 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Dec 2023 14:00:52 +0000 Subject: [PATCH 097/116] Better clickhouse-format: support VALUES, comments --- .../operations/utilities/clickhouse-format.md | 2 + programs/format/Format.cpp | 149 +++++++++++++----- .../0_stateless/02946_format_values.reference | 139 ++++++++++++++++ .../0_stateless/02946_format_values.sh | 73 +++++++++ 4 files changed, 321 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/02946_format_values.reference create mode 100755 tests/queries/0_stateless/02946_format_values.sh diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index 3e4295598aa..879bf9d71ac 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -11,6 +11,8 @@ Keys: - `--query` — Format queries of any length and complexity. - `--hilite` — Add syntax highlight with ANSI terminal escape sequences. - `--oneline` — Format in single line. +- `--max_line_length` — Format in single line queries with length less than specified. +- `--comments` — Keep comments in the output. - `--quiet` or `-q` — Just check syntax, no output on success. - `--multiquery` or `-n` — Allow multiple queries in the same file. - `--obfuscate` — Obfuscate instead of formatting. diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 05ba86069d7..f7385335de9 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -30,21 +30,49 @@ #include #include +#include +#include + +namespace DB::ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +namespace +{ + +void skipSpacesAndComments(const char*& pos, const char* end, bool print_comments) +{ + do + { + /// skip spaces to avoid throw exception after last query + while (pos != end && std::isspace(*pos)) + ++pos; + + const char * comment_begin = pos; + /// for skip comment after the last query and to not throw exception + if (end - pos > 2 && *pos == '-' && *(pos + 1) == '-') + { + pos += 2; + /// skip until the end of the line + while (pos != end && *pos != '\n') + ++pos; + if (print_comments) + std::cout << std::string_view(comment_begin, pos - comment_begin) << "\n"; + } + /// need to parse next sql + else + break; + } while (pos != end); +} + +} #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" extern const char * auto_time_zones[]; - -namespace DB -{ -namespace ErrorCodes -{ -extern const int INVALID_FORMAT_INSERT_QUERY_WITH_DATA; -} -} - int mainEntryClickHouseFormat(int argc, char ** argv) { using namespace DB; @@ -55,8 +83,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv) desc.add_options() ("query", po::value(), "query to format") ("help,h", "produce help message") + ("comments", "keep comments in the output") ("hilite", "add syntax highlight with ANSI terminal escape sequences") ("oneline", "format in single line") + ("max_line_length", po::value()->default_value(0), "format in single line queries with length less than specified") ("quiet,q", "just check syntax, no output on success") ("multiquery,n", "allow multiple queries in the same file") ("obfuscate", "obfuscate instead of formatting") @@ -88,6 +118,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) bool oneline = options.count("oneline"); bool quiet = options.count("quiet"); bool multiple = options.count("multiquery"); + bool print_comments = options.count("comments"); + size_t max_line_length = options["max_line_length"].as(); bool obfuscate = options.count("obfuscate"); bool backslash = options.count("backslash"); bool allow_settings_after_format_in_insert = options.count("allow_settings_after_format_in_insert"); @@ -104,6 +136,19 @@ int mainEntryClickHouseFormat(int argc, char ** argv) return 2; } + if (oneline && max_line_length) + { + std::cerr << "Options 'oneline' and 'max_line_length' are mutually exclusive." << std::endl; + return 2; + } + + if (max_line_length > 255) + { + std::cerr << "Option 'max_line_length' must be less than 256." << std::endl; + return 2; + } + + String query; if (options.count("query")) @@ -124,7 +169,6 @@ int mainEntryClickHouseFormat(int argc, char ** argv) if (options.count("seed")) { - std::string seed; hash_func.update(options["seed"].as()); } @@ -179,30 +223,68 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { const char * pos = query.data(); const char * end = pos + query.size(); + skipSpacesAndComments(pos, end, print_comments); ParserQuery parser(end, allow_settings_after_format_in_insert); - do + while (pos != end) { + size_t approx_query_length = multiple ? find_first_symbols<';'>(pos, end) - pos : end - pos; + ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, - /// we should throw an exception early, and make exception message more readable. - if (const auto * insert_query = res->as(); insert_query && insert_query->data) + std::unique_ptr insert_query_payload = nullptr; + /// If the query is INSERT ... VALUES, then we will try to parse the data. + if (auto * insert_query = res->as(); insert_query && insert_query->data) { - throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, - "Can't format ASTInsertQuery with data, since data will be lost"); + if ("Values" != insert_query->format) + throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED, + "Can't format INSERT query with data format '{}'", insert_query->format); + + /// We assume that data ends with a newline character (same as client does) + const char * this_query_end = find_first_symbols<'\n'>(insert_query->data, end); + insert_query->end = this_query_end; + pos = this_query_end; + insert_query_payload = getReadBufferFromASTInsertQuery(res); } if (!quiet) { if (!backslash) { - WriteBufferFromOStream res_buf(std::cout, 4096); - formatAST(*res, res_buf, hilite, oneline); - res_buf.finalize(); - if (multiple) - std::cout << "\n;\n"; + WriteBufferFromOwnString str_buf; + formatAST(*res, str_buf, hilite, oneline || approx_query_length < max_line_length); + + if (insert_query_payload) + { + str_buf.write(' '); + copyData(*insert_query_payload, str_buf); + if (multiple) + str_buf.write('\n'); + } + + String res_string = str_buf.str(); + const char * s_pos = res_string.data(); + const char * s_end = s_pos + res_string.size(); + WriteBufferFromOStream res_cout(std::cout, 4096); + /// For multiline queries we print ';' at new line, + /// but for single line queries we print ';' at the same line + bool has_multiple_lines = false; + while (s_pos != s_end) + { + if (*s_pos == '\n') + has_multiple_lines = true; + res_cout.write(*s_pos++); + } + res_cout.finalize(); + + if (multiple && !insert_query_payload) + { + if (oneline || !has_multiple_lines) + std::cout << ";\n"; + else + std::cout << "\n;\n"; + } std::cout << std::endl; } /// add additional '\' at the end of each line; @@ -230,27 +312,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv) std::cout << std::endl; } } - - do - { - /// skip spaces to avoid throw exception after last query - while (pos != end && std::isspace(*pos)) - ++pos; - - /// for skip comment after the last query and to not throw exception - if (end - pos > 2 && *pos == '-' && *(pos + 1) == '-') - { - pos += 2; - /// skip until the end of the line - while (pos != end && *pos != '\n') - ++pos; - } - /// need to parse next sql - else - break; - } while (pos != end); - - } while (multiple && pos != end); + skipSpacesAndComments(pos, end, print_comments); + if (!multiple) + break; + } } } catch (...) diff --git a/tests/queries/0_stateless/02946_format_values.reference b/tests/queries/0_stateless/02946_format_values.reference new file mode 100644 index 00000000000..8f896fd3ba0 --- /dev/null +++ b/tests/queries/0_stateless/02946_format_values.reference @@ -0,0 +1,139 @@ +INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td') +====================================== +SELECT a +FROM table1 +; + +INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); + +SELECT b +FROM table1 +; + +====================================== +-- begin +SELECT a +FROM table1 +; + +-- some insert query +INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); + +-- more comments +-- in a row +SELECT b +FROM table1 +; + +-- end +====================================== +SELECT b FROM table1; + +SELECT b, c FROM table1; + +SELECT + b, + c, + d +FROM table1 +; + +SELECT + b, + c, + d, + e +FROM table1 +; + +SELECT + b, + c, + d, + e, + f +FROM table1 +; + +SELECT + b, + c +FROM +( + SELECT + b, + c + FROM table1 +) +; + +SELECT + b, + c, + d, + e, + f +FROM +( + SELECT + b, + c, + d, + e, + f + FROM table1 +) +; + +====================================== +SELECT b FROM table1; + +SELECT b, c FROM table1; + +SELECT b, c, d FROM table1; + +SELECT b, c, d, e FROM table1; + +SELECT b, c, d, e, f FROM table1; + +SELECT b, c FROM (SELECT b, c FROM table1); + +SELECT + b, + c, + d, + e, + f +FROM +( + SELECT + b, + c, + d, + e, + f + FROM table1 +) +; + +====================================== +SELECT + b, + c, + d, + e, + f +FROM +( + SELECT + b, + c, + d, + e, + f + FROM table1 +) +SELECT b, c, d, e, f FROM (SELECT b, c, d, e, f FROM table1) +====================================== +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02946_format_values.sh b/tests/queries/0_stateless/02946_format_values.sh new file mode 100755 index 00000000000..cd44d31d1a1 --- /dev/null +++ b/tests/queries/0_stateless/02946_format_values.sh @@ -0,0 +1,73 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "insert into table1 values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td')" | ${CLICKHOUSE_FORMAT} + +echo "======================================" + +cat <&1 | grep -o 'BAD_ARGUMENTS' ||: +echo "select 1" | ${CLICKHOUSE_FORMAT} --comments --max_line_length=120 --oneline 2>&1 | grep -o 'BAD_ARGUMENTS' ||: From cccc24c2fd684bd7a00281af33960c348355d477 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Dec 2023 17:13:56 +0000 Subject: [PATCH 098/116] upd --- programs/format/Format.cpp | 14 ++++++++++---- .../01753_fix_clickhouse_format.reference | 8 ++++---- .../0_stateless/01753_fix_clickhouse_format.sh | 4 +++- .../02263_format_insert_settings.reference | 2 +- .../0_stateless/02263_format_insert_settings.sh | 2 +- .../0_stateless/02946_format_values.reference | 12 +++++++----- tests/queries/0_stateless/02946_format_values.sh | 4 ++-- 7 files changed, 28 insertions(+), 18 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index f7385335de9..74306d742f2 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include #include @@ -29,9 +31,8 @@ #include #include #include - #include -#include + namespace DB::ErrorCodes { @@ -238,8 +239,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv) if (auto * insert_query = res->as(); insert_query && insert_query->data) { if ("Values" != insert_query->format) - throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED, - "Can't format INSERT query with data format '{}'", insert_query->format); + throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Can't format INSERT query with data format '{}'", insert_query->format); + + /// Reset format to default to have `INSERT INTO table VALUES` instead of `INSERT INTO table VALUES FORMAT Values` + insert_query->format = {}; /// We assume that data ends with a newline character (same as client does) const char * this_query_end = find_first_symbols<'\n'>(insert_query->data, end); @@ -266,6 +269,9 @@ int mainEntryClickHouseFormat(int argc, char ** argv) String res_string = str_buf.str(); const char * s_pos = res_string.data(); const char * s_end = s_pos + res_string.size(); + /// remove trailing spaces + while (s_end > s_pos && isWhitespaceASCIIOneLine(*(s_end - 1))) + --s_end; WriteBufferFromOStream res_cout(std::cout, 4096); /// For multiline queries we print ';' at new line, /// but for single line queries we print ';' at the same line diff --git a/tests/queries/0_stateless/01753_fix_clickhouse_format.reference b/tests/queries/0_stateless/01753_fix_clickhouse_format.reference index 0aad4d64c55..735d4099534 100644 --- a/tests/queries/0_stateless/01753_fix_clickhouse_format.reference +++ b/tests/queries/0_stateless/01753_fix_clickhouse_format.reference @@ -1,5 +1,4 @@ -SELECT 1 -; +SELECT 1; SELECT 1 UNION ALL @@ -10,8 +9,7 @@ UNION ALL ) ; -SELECT 1 -; +SELECT 1; SELECT 1 UNION ALL @@ -22,4 +20,6 @@ UNION ALL ) ; +INSERT INTO t VALUES (1); + OK diff --git a/tests/queries/0_stateless/01753_fix_clickhouse_format.sh b/tests/queries/0_stateless/01753_fix_clickhouse_format.sh index 5cdd53b2166..ba7fe949833 100755 --- a/tests/queries/0_stateless/01753_fix_clickhouse_format.sh +++ b/tests/queries/0_stateless/01753_fix_clickhouse_format.sh @@ -8,4 +8,6 @@ echo "select 1; select 1 union all (select 1 union distinct select 1); " | $CL echo "select 1; select 1 union all (select 1 union distinct select 1); -- comment " | $CLICKHOUSE_FORMAT -n; -echo "insert into t values (1); " | $CLICKHOUSE_FORMAT -n 2>&1 \ | grep -F -q "Code: 578" && echo 'OK' || echo 'FAIL' +echo "insert into t values (1); " | $CLICKHOUSE_FORMAT -n + +echo 'insert into t format JSONEachRow {"a":1};' | $CLICKHOUSE_FORMAT -n 2>&1 \ | grep -F -q "NOT_IMPLEMENTED" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index e2d1ec3980e..2bba75f6788 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -1,7 +1,7 @@ [multi] insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): [multi] insert into foo format tsv settings max_threads=1 -Can't format ASTInsertQuery with data, since data will be lost. +NOT_IMPLEMENTED [multi] insert into foo format tsv settings max_threads=1 INSERT INTO foo SETTINGS max_threads = 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 8b156ffec83..49aa56d6c0a 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -25,7 +25,7 @@ function run_format_both() run_format 'insert into foo settings max_threads=1' |& grep --max-count 2 --only-matching -e "Syntax error (query): failed at position .* (end of query):" -e '^\[.*$' # compatibility -run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "Can't format ASTInsertQuery with data, since data will be lost." -e '^\[.*$' +run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "NOT_IMPLEMENTED" -e '^\[.*$' run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert |& grep --max-count 2 --only-matching -e "You have SETTINGS before and after FORMAT" -e '^\[.*$' diff --git a/tests/queries/0_stateless/02946_format_values.reference b/tests/queries/0_stateless/02946_format_values.reference index 8f896fd3ba0..90b2a3cb8ef 100644 --- a/tests/queries/0_stateless/02946_format_values.reference +++ b/tests/queries/0_stateless/02946_format_values.reference @@ -1,10 +1,10 @@ -INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td') +INSERT INTO table1 VALUES (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td') ====================================== SELECT a FROM table1 ; -INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); +INSERT INTO table1 VALUES (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); SELECT b FROM table1 @@ -17,7 +17,7 @@ FROM table1 ; -- some insert query -INSERT INTO table1 FORMAT Values (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); +INSERT INTO table1 VALUES (1, [1,3], 'fd'), (2, [2,4], 'sd'), (3, [3,5], 'td'); -- more comments -- in a row @@ -135,5 +135,7 @@ FROM ) SELECT b, c, d, e, f FROM (SELECT b, c, d, e, f FROM table1) ====================================== -BAD_ARGUMENTS -BAD_ARGUMENTS +Option 'max_line_length' must be less than 256. +2 +Options 'oneline' and 'max_line_length' are mutually exclusive. +2 diff --git a/tests/queries/0_stateless/02946_format_values.sh b/tests/queries/0_stateless/02946_format_values.sh index cd44d31d1a1..36e32de42fa 100755 --- a/tests/queries/0_stateless/02946_format_values.sh +++ b/tests/queries/0_stateless/02946_format_values.sh @@ -69,5 +69,5 @@ echo "select b, c, d, e, f from ( select b, c, d, e, f from table1 )" | ${CLICKH echo "======================================" -echo "select 1" | ${CLICKHOUSE_FORMAT} --comments --max_line_length=260 2>&1 | grep -o 'BAD_ARGUMENTS' ||: -echo "select 1" | ${CLICKHOUSE_FORMAT} --comments --max_line_length=120 --oneline 2>&1 | grep -o 'BAD_ARGUMENTS' ||: +{ echo "select 1" | ${CLICKHOUSE_FORMAT} --comments --max_line_length=260 2>&1; echo $?; } +{ echo "select 1" | ${CLICKHOUSE_FORMAT} --comments --max_line_length=120 --oneline 2>&1; echo $?; } From e68fc2df7a3ddd357e80f2f3ea4fecccc62732c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 12:16:53 +0000 Subject: [PATCH 099/116] Remove spaces after insert in clickhouse-format --- programs/format/Format.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 74306d742f2..9f294312ee3 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -262,8 +262,6 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { str_buf.write(' '); copyData(*insert_query_payload, str_buf); - if (multiple) - str_buf.write('\n'); } String res_string = str_buf.str(); @@ -291,6 +289,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv) else std::cout << "\n;\n"; } + else if (multiple && insert_query_payload) + /// Do not need to add ; because it's already in the insert_query_payload + std::cout << "\n"; + std::cout << std::endl; } /// add additional '\' at the end of each line; From 9bd48512238c0eb7d8f2bfbaaaf674e31525a669 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jan 2024 15:35:02 +0100 Subject: [PATCH 100/116] fix 02675_profile_events_from_query_log_and_client --- .../02675_profile_events_from_query_log_and_client.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index a97879eaca8..ffb3f461f1c 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,4 +1,5 @@ INSERT TO S3 + [ 0 ] S3Clients: 1 [ 0 ] S3CompleteMultipartUpload: 1 [ 0 ] S3CreateMultipartUpload: 1 [ 0 ] S3HeadObject: 2 From 55ec1a17e76260293d805ea35c7d6a5da29db743 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jan 2024 15:58:47 +0100 Subject: [PATCH 101/116] remove debug logging --- src/Common/MatchGenerator.cpp | 23 ++----------------- src/Common/MatchGenerator.h | 2 +- src/Common/ObjectStorageKeyGenerator.cpp | 2 +- .../tests/gtest_generate_random_by_regexp.cpp | 6 ++--- 4 files changed, 7 insertions(+), 26 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 5495d58d65f..f047c21b470 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -24,7 +24,6 @@ #include "MatchGenerator.h" #include -#include #include #include #include @@ -330,13 +329,6 @@ private: public: - explicit RandomStringPrepareWalker(bool logging) - : logger(logging ? &Poco::Logger::get("GeneratorCombiner") : nullptr) - { - if (logger) - LOG_DEBUG(logger, "GeneratorCombiner"); - } - std::function getGenerator() { if (root == nullptr) @@ -374,16 +366,11 @@ private: Regexp * ShortVisit(Regexp* /*re*/, Regexp * /*parent_arg*/) override { - if (logger) - LOG_DEBUG(logger, "ShortVisit"); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ShortVisit should not be called"); } Regexp * PreVisit(Regexp * re, Regexp * parent_arg, bool* /*stop*/) override /*noexcept*/ { - if (logger) - LOG_DEBUG(logger, "GeneratorCombiner PreVisit node {}", magic_enum::enum_name(re->op())); - if (parent_arg == nullptr) { chassert(root == nullptr); @@ -397,10 +384,6 @@ private: Regexp * PostVisit(Regexp * re, Regexp * /*parent_arg*/, Regexp * pre_arg, Regexp ** child_args, int nchild_args) override /*noexcept*/ { - if (logger) - LOG_DEBUG(logger, "GeneratorCombiner PostVisit node {}", - magic_enum::enum_name(re->op())); - switch (re->op()) { case kRegexpConcat: // Matches concatenation of sub_[0..nsub-1]. @@ -456,8 +439,6 @@ private: return pre_arg; } - Poco::Logger * logger = nullptr; - Regexp * root = nullptr; Generators generators; }; @@ -473,7 +454,7 @@ void RandomStringGeneratorByRegexp::RegexpPtrDeleter::operator() (re2::Regexp * re->Decref(); } -RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(const String & re_str, bool logging) +RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(const String & re_str) { re2::RE2::Options options; options.set_case_sensitive(true); @@ -490,7 +471,7 @@ RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(const String & re_s regexp.reset(regexp->Simplify()); - auto walker = re2::RandomStringPrepareWalker(logging); + auto walker = re2::RandomStringPrepareWalker(); walker.Walk(regexp.get(), {}); generatorFunc = walker.getGenerator(); diff --git a/src/Common/MatchGenerator.h b/src/Common/MatchGenerator.h index 65bfe0c9208..68b22404d5a 100644 --- a/src/Common/MatchGenerator.h +++ b/src/Common/MatchGenerator.h @@ -14,7 +14,7 @@ namespace DB class RandomStringGeneratorByRegexp { public: - RandomStringGeneratorByRegexp(const String & re_str, bool logging); + explicit RandomStringGeneratorByRegexp(const String & re_str); String generate() const; private: diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index c970b193adb..7b4507a3abc 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -11,7 +11,7 @@ class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator public: explicit GeneratorWithTemplate(String key_template_) : key_template(std::move(key_template_)) - , re_gen(key_template, /*logging*/ false) + , re_gen(key_template) { } DB::ObjectStorageKey generate(const String &) const override diff --git a/src/Common/tests/gtest_generate_random_by_regexp.cpp b/src/Common/tests/gtest_generate_random_by_regexp.cpp index 67ccf1ffe84..2f6260891c6 100644 --- a/src/Common/tests/gtest_generate_random_by_regexp.cpp +++ b/src/Common/tests/gtest_generate_random_by_regexp.cpp @@ -8,7 +8,7 @@ void routine(String s) { std::cerr << "case '"<< s << "'"; - auto gen = DB::RandomStringGeneratorByRegexp(s, /*logging*/ true); + auto gen = DB::RandomStringGeneratorByRegexp(s); [[maybe_unused]] auto res = gen.generate(); std::cerr << " result '"<< res << "'" << std::endl; } @@ -46,7 +46,7 @@ TEST(GenerateRandomString, Negative) TEST(GenerateRandomString, DifferentResult) { std::cerr << "100 different keys" << std::endl; - auto gen = DB::RandomStringGeneratorByRegexp("prefix-[a-z]{3}-suffix/[0-9a-f]{20}", /*logging*/ true); + auto gen = DB::RandomStringGeneratorByRegexp("prefix-[a-z]{3}-suffix/[0-9a-f]{20}"); std::set deduplicate; for (int i = 0; i < 100; ++i) ASSERT_TRUE(deduplicate.insert(gen.generate()).second); @@ -56,7 +56,7 @@ TEST(GenerateRandomString, DifferentResult) TEST(GenerateRandomString, FullRange) { std::cerr << "all possible letters" << std::endl; - auto gen = DB::RandomStringGeneratorByRegexp("[a-z]", /*logging*/ false); + auto gen = DB::RandomStringGeneratorByRegexp("[a-z]"); std::set deduplicate; int count = 'z' - 'a' + 1; while (deduplicate.size() < count) From cf53d9a17ca8d9756b9d2784a8b7396af5f3f326 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 8 Jan 2024 11:05:36 -0500 Subject: [PATCH 102/116] add USE_QAT into build_options --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 2 +- src/Compression/CompressionCodecZSTDQAT.cpp | 22 +++++++++++-------- .../System/StorageSystemBuildOptions.cpp.in | 1 + 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 45a956dcb55..72d21a8572b 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -27,7 +27,7 @@ if (ENABLE_QAT_OUT_OF_TREE_BUILD) ${QAT_AL_INCLUDE_DIR} ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_QATZSTD_COMPRESSION) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) else () # In-tree build message(STATUS "Intel QATZSTD in-tree build") diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index e3a6714f191..f7ff783bf9c 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -86,17 +86,21 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) cctx = ZSTD_createCCtx(); int res = QZSTD_startQatDevice(); - LOG_DEBUG(log, "Initialization of ZSTD_QAT codec, status: {} ", res); - sequenceProducerState = QZSTD_createSeqProdState(); + if(res == QZSTD_OK) + { + sequenceProducerState = QZSTD_createSeqProdState(); + ZSTD_registerSequenceProducer( + cctx, + sequenceProducerState, + qatSequenceProducer + ); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + LOG_DEBUG(log, "Hardware-assisted ZSTD_QAT codec is ready!"); + } + else + LOG_DEBUG(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, status: {} - please refer to QZSTD_Status_e in ./contrib/QAT-ZSTD-Plugin/src/qatseqprod.h", res); - ZSTD_registerSequenceProducer( - cctx, - sequenceProducerState, - qatSequenceProducer - ); - - ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); } diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 796b134ba56..a81bcb08bfc 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -63,6 +63,7 @@ const char * auto_config_build[] "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", "USE_QPL", "@ENABLE_QPL@", + "USE_QAT", "@ENABLE_QATLIB@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", From 378602e1c8e3b3525f466ec08563442bfa16cf76 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Mon, 8 Jan 2024 18:29:41 +0100 Subject: [PATCH 103/116] Update 02675_profile_events_from_query_log_and_client.reference --- .../02675_profile_events_from_query_log_and_client.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index ffb3f461f1c..babcecf7004 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,5 +1,5 @@ INSERT TO S3 - [ 0 ] S3Clients: 1 + [ 0 ] S3Clients: 1 [ 0 ] S3CompleteMultipartUpload: 1 [ 0 ] S3CreateMultipartUpload: 1 [ 0 ] S3HeadObject: 2 From 9b6c8cd44fb8926614490da71df68631655952fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jan 2024 20:27:16 +0300 Subject: [PATCH 104/116] Fix possible NULL dereference during symbolizing inline frames It is possible sometimes:
``` 2024.01.02 15:18:49.542342 [ 102142 ] {} BaseDaemon: Address: NULL pointer. Access: read. Address not mapped to object. 2024.01.02 15:18:49.542356 [ 102142 ] {} BaseDaemon: Stack trace: 0x0000000012817007 0x00000000120b3e88 0x000000001209c3e8 0x000000001209b7f3 0x000000001213d936 0x0000000011f71042 0x0000000011f6dd22 0x0000 000012496116 0x000000001249181c 0x00000000133197ec 0x000000001332bc79 0x0000000015d0eb14 0x0000000015d0fd11 0x0000000015e1b367 0x0000000015e195fc 0x00007f011cb31fa3 0x00007f011ca624cf 2024.01.02 15:18:50.052773 [ 102142 ] {} BaseDaemon: 2.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:603: shared_ptr 2024.01.02 15:18:50.052829 [ 102142 ] {} BaseDaemon: 2.2. inlined from ./build_docker/./src/Planner/PlannerContext.h:56: DB::PlannerContext::getQueryContext() const 2024.01.02 15:18:50.052853 [ 102142 ] {} BaseDaemon: 2.3. inlined from ./build_docker/./src/Storages/StorageDistributed.cpp:666: DB::(anonymous namespace)::buildQueryTreeDistributed(DB::SelectQueryInfo&, s td::shared_ptr const&, DB::StorageID const&, std::shared_ptr const&) 2024.01.02 15:18:50.052872 [ 102142 ] {} BaseDaemon: 2. ./build_docker/./src/Storages/StorageDistributed.cpp:743: DB::StorageDistributed::read(DB::QueryPlan&, std::vector> co nst&, std::shared_ptr const&, DB::SelectQueryInfo&, std::shared_ptr, DB::QueryProcessingStage::Enum, unsigned long, unsigned long) @ 0x0000000012817007 in /usr/lib/debug/usr /bin/clickhouse.debug 2024.01.02 15:18:50.211769 [ 102142 ] {} BaseDaemon: 3.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:815: std::shared_ptr::operator->[abi:v15000]() const 2024.01.02 15:18:50.211815 [ 102142 ] {} BaseDaemon: 3. ./build_docker/./src/Interpreters/InterpreterSelectQuery.cpp:2488: DB::InterpreterSelectQuery::executeFetchColumns(DB::QueryProcessingStage::Enum, DB ::QueryPlan&) @ 0x00000000120b3e88 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.295382 [ 102142 ] {} BaseDaemon: 4. ./build_docker/./src/Interpreters/InterpreterSelectQuery.cpp:1444: DB::InterpreterSelectQuery::executeImpl(DB::QueryPlan&, std::optional) @ 0x000000001209c3e8 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.375779 [ 102142 ] {} BaseDaemon: 5.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/optional:260: ~__optional_destruct_base 2024.01.02 15:18:50.375820 [ 102142 ] {} BaseDaemon: 5. ./build_docker/./src/Interpreters/InterpreterSelectQuery.cpp:868: DB::InterpreterSelectQuery::buildQueryPlan(DB::QueryPlan&) @ 0x000000001209b7f3 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.403975 [ 102142 ] {} BaseDaemon: 6. ./build_docker/./src/Interpreters/InterpreterSelectWithUnionQuery.cpp:0: DB::InterpreterSelectWithUnionQuery::buildQueryPlan(DB::QueryPlan&) @ 0x0000 00001213d936 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.432051 [ 102142 ] {} BaseDaemon: 7.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:603: shared_ptr 2024.01.02 15:18:50.432472 [ 102143 ] {} BaseDaemon: ########## Short fault info ############ 2024.01.02 15:18:50.432500 [ 102143 ] {} BaseDaemon: (version 23.9.2.56 (official build), build id: 76109A79FA62B9BC630A6C39438DEA7D28147B68, git hash: a1bf3f1de55abf2354dc498ffbee270be043d633) (from threa d 102142) Received signal 11 2024.01.02 15:18:50.432516 [ 102143 ] {} BaseDaemon: Signal description: Segmentation fault 2024.01.02 15:18:50.432526 [ 102143 ] {} BaseDaemon: Address: NULL pointer. Access: read. Address not mapped to object. 2024.01.02 15:18:50.432539 [ 102143 ] {} BaseDaemon: Stack trace: 0x00007f011cac5181 0x0000000015ccd934 0x000000000c76771e 0x000000000ca0fe32 0x000000000ca0ccf5 0x00007f011cb31fa3 0x00007f011ca624cf 2024.01.02 15:18:50.432547 [ 102143 ] {} BaseDaemon: ######################################## 2024.01.02 15:18:50.432565 [ 102143 ] {} BaseDaemon: (version 23.9.2.56 (official build), build id: 76109A79FA62B9BC630A6C39438DEA7D28147B68, git hash: a1bf3f1de55abf2354dc498ffbee270be043d633) (from threa d 102142) (no query) Received signal Segmentation fault (11) ``` ``` 2024.01.02 15:18:50.432588 [ 102143 ] {} BaseDaemon: Address: NULL pointer. Access: read. Address not mapped to object. 2024.01.02 15:18:50.432601 [ 102143 ] {} BaseDaemon: Stack trace: 0x00007f011cac5181 0x0000000015ccd934 0x000000000c76771e 0x000000000ca0fe32 0x000000000ca0ccf5 0x00007f011cb31fa3 0x00007f011ca624cf 2024.01.02 15:18:50.432638 [ 102143 ] {} BaseDaemon: 2. ? @ 0x000000000015c181 in /usr/lib/x86_64-linux-gnu/libc-2.28.so 2024.01.02 15:18:50.446953 [ 102143 ] {} BaseDaemon: 3.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/string:1955: String::__init(char const*, unsigned long) 2024.01.02 15:18:50.446981 [ 102143 ] {} BaseDaemon: 3.2. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/string:843: basic_string 2024.01.02 15:18:50.446998 [ 102143 ] {} BaseDaemon: 3. ./build_docker/./base/base/demangle.cpp:25: demangle(char const*, int&) @ 0x0000000015ccd934 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.458431 [ 102143 ] {} BaseDaemon: 4. ./build_docker/./src/Common/StackTrace.cpp:389: void toStringEveryLineImpl>)>>(bool, StackTraceRefTriple const&, std::function>)>&&) (.llvm.10713299015003964940) @ 0x000000000c76771e in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.474955 [ 102143 ] {} BaseDaemon: 5.1. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:818: ~__policy_func 2024.01.02 15:18:50.474997 [ 102143 ] {} BaseDaemon: 5.2. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1174: ~function 2024.01.02 15:18:50.475010 [ 102143 ] {} BaseDaemon: 5. ./build_docker/./src/Daemon/BaseDaemon.cpp:415: SignalListener::onFault(int, siginfo_t const&, ucontext_t*, StackTrace const&, std::vector, std::allocator>> const&, unsigned int, DB::ThreadStatus*) const @ 0x000000000ca0fe32 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.512270 [ 102143 ] {} BaseDaemon: 6.1. inlined from ./build_docker/./src/Daemon/BaseDaemon.cpp:284: operator() 2024.01.02 15:18:50.512591 [ 102143 ] {} BaseDaemon: 6.2. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394: decltype(std::declval()()) std::__invoke[abi:v15000](SignalListener::run()::'lambda'()&&) 2024.01.02 15:18:50.512648 [ 102143 ] {} BaseDaemon: 6.3. inlined from ./build_docker/./contrib/llvm-project/libcxx/include/thread:284: void std::__thread_execute[abi:v15000]>, SignalListener::run()::'lambda'()>(std::tuple>, SignalListener::run()::'lambda'()>&, std::__tuple_indices<>) 2024.01.02 15:18:50.512664 [ 102143 ] {} BaseDaemon: 6. ./build_docker/./contrib/llvm-project/libcxx/include/thread:295: void* std::__thread_proxy[abi:v15000]>, SignalListener::run()::'lambda'()>>(void*) @ 0x000000000ca0ccf5 in /usr/lib/debug/usr/bin/clickhouse.debug 2024.01.02 15:18:50.512698 [ 102143 ] {} BaseDaemon: 7. start_thread @ 0x0000000000007fa3 in /usr/lib/x86_64-linux-gnu/libpthread-2.28.so 2024.01.02 15:18:50.512721 [ 102143 ] {} BaseDaemon: 8. ? @ 0x00000000000f94cf in /usr/lib/x86_64-linux-gnu/libc-2.28.so 2024.01.02 15:18:50.666148 [ 102143 ] {} BaseDaemon: Integrity check of the executable successfully passed (checksum: 3A99FBFAA2DA54D46975E9ABC04E53E0) 2024.01.02 15:18:50.746677 [ 102143 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues ``` **The problem is actually here** - [1]. Though to be precise here - [2]. See also - [3]. [1]: https://github.com/ClickHouse/ClickHouse/blob/5e467b69c8f212b07375dd3008bcc02afbb64357/src/Common/StackTrace.cpp#L405 [2]: https://github.com/azat/ClickHouse/blob/4d734cf1e5bc764024945209973ab1f317016932/src/Common/StackTrace.cpp#L389 [3]: https://github.com/ClickHouse/ClickHouse/blob/5e467b69c8f212b07375dd3008bcc02afbb64357/src/Common/Dwarf.cpp#L1161 _(It is not obvious from the patch, so will post a snippet)_
Signed-off-by: Azat Khuzhin --- src/Common/StackTrace.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 21235914f7c..fe513199ac2 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -296,6 +296,9 @@ constexpr std::pair replacements[] // Replace parts from @c replacements with shorter aliases String demangleAndCollapseNames(std::string_view file, const char * const symbol_name) { + if (!symbol_name) + return "?"; + std::string_view file_copy = file; if (auto trim_pos = file.find_last_of('/'); trim_pos != file.npos) file_copy.remove_suffix(file.size() - trim_pos); From f67edfb397d96fde056086f963a51fe1e0cae6a8 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Mon, 8 Jan 2024 18:42:33 +0000 Subject: [PATCH 105/116] Consume leading zeroes - use find_first_not_symbols --- src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index fe11e3f6360..8c18df97c46 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -603,10 +603,8 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType( memcpy(buf, istr.position(), bytes_to_copy); buf[bytes_to_copy] = 0; - /// Consume leading zeroes - we don't want any funny octal business - auto* non_zero_buf = buf; - while (*non_zero_buf == '0') - ++non_zero_buf; + /// Skip leading zeroes - we don't want any funny octal business + auto* non_zero_buf = find_first_not_symbols<'0'>(buf, buf + bytes_to_copy); char * pos_double = non_zero_buf; errno = 0; From 5210bbc5876aea3689a9c7c96a59c40ad1bc5c71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 9 Jan 2024 05:13:52 +0300 Subject: [PATCH 106/116] Update ConstantExpressionTemplate.cpp --- src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 8c18df97c46..316a84fe94f 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -604,7 +604,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType( buf[bytes_to_copy] = 0; /// Skip leading zeroes - we don't want any funny octal business - auto* non_zero_buf = find_first_not_symbols<'0'>(buf, buf + bytes_to_copy); + char * non_zero_buf = find_first_not_symbols<'0'>(buf, buf + bytes_to_copy); char * pos_double = non_zero_buf; errno = 0; From 37a7ce29504eaf96546e8ba1038f6c30c1e9baaf Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 9 Jan 2024 17:17:30 +0800 Subject: [PATCH 107/116] Apply suggestions from code review Co-authored-by: pufit --- docs/en/operations/settings/settings.md | 4 ++-- src/Core/Settings.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0cf6dbd80a1..0c7b9950972 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4783,9 +4783,9 @@ Default: `3` ## output_format_compression_zstd_window_log -To be used when output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression. This can help to achieve better compression ratio. +Can be used when the output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (power of `2`) and enables a long-range mode for zstd compression. This can help to achieve a better compression ratio. -Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`). +Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw an exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`). Default: `0` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 79a1c3a968b..68bffe9f801 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -205,7 +205,7 @@ class IColumn; M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \ M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table function `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`.", 0) \ - M(UInt64, output_format_compression_zstd_window_log, 0, "To be used when output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (expressed as power of `2`) and enable long-range mode for zstd compression.", 0) \ + M(UInt64, output_format_compression_zstd_window_log, 0, "Can be used when the output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (power of `2`) and enables a long-range mode for zstd compression.", 0) \ \ M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ From 0e10356b9547ed4d53ae296c4711a2b73b71f4f1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 9 Jan 2024 09:46:31 +0000 Subject: [PATCH 108/116] randomize filename to avoid flaky test Signed-off-by: Duc Canh Le --- .../02961_output_format_compress_params.reference | 1 + .../02961_output_format_compress_params.sh | 13 +++++++++++++ .../02961_output_format_compress_params.sql | 4 ---- 3 files changed, 14 insertions(+), 4 deletions(-) create mode 100755 tests/queries/0_stateless/02961_output_format_compress_params.sh delete mode 100644 tests/queries/0_stateless/02961_output_format_compress_params.sql diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.reference b/tests/queries/0_stateless/02961_output_format_compress_params.reference index 749fce669df..d0752a77fc7 100644 --- a/tests/queries/0_stateless/02961_output_format_compress_params.reference +++ b/tests/queries/0_stateless/02961_output_format_compress_params.reference @@ -1 +1,2 @@ +1 1000000 diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.sh b/tests/queries/0_stateless/02961_output_format_compress_params.sh new file mode 100755 index 00000000000..7275f9a0b2b --- /dev/null +++ b/tests/queries/0_stateless/02961_output_format_compress_params.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: replica + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +file_with_random_postfix=test_02961_`date +%s%6N`.csv + +${CLICKHOUSE_CLIENT} --query "INSERT INTO FUNCTION file('${file_with_random_postfix}', 'CSV', 'x UInt64', 'zstd') SELECT number FROM numbers(1000000) SETTINGS output_format_compression_level = 10, output_format_compression_zstd_window_log = 30, engine_file_truncate_on_insert = 1;" +# Simple check that output_format_compression_zstd_window_log = 30 works +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM file('${file_with_random_postfix}', 'CSV', 'x UInt64', 'zstd') SETTINGS zstd_window_log_max = 29;" 2>&1 | head -n 1 | grep -c "ZSTD_DECODER_FAILED" +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM file('${file_with_random_postfix}', 'CSV', 'x UInt64', 'zstd') SETTINGS zstd_window_log_max = 30;" diff --git a/tests/queries/0_stateless/02961_output_format_compress_params.sql b/tests/queries/0_stateless/02961_output_format_compress_params.sql deleted file mode 100644 index fc67bca5816..00000000000 --- a/tests/queries/0_stateless/02961_output_format_compress_params.sql +++ /dev/null @@ -1,4 +0,0 @@ -INSERT INTO FUNCTION file('test_02961.csv', 'CSV', 'x UInt64', 'zstd') SELECT number FROM numbers(1000000) SETTINGS output_format_compression_level = 10, output_format_compression_zstd_window_log = 30, engine_file_truncate_on_insert = 1; --- Simple check that output_format_compression_zstd_window_log = 30 works -SELECT count() FROM file('test_02961.csv', 'CSV', 'x UInt64', 'zstd'); -- { serverError ZSTD_DECODER_FAILED } -SELECT count() FROM file('test_02961.csv', 'CSV', 'x UInt64', 'zstd') SETTINGS zstd_window_log_max = 30; From ed8eb8f497975c39cd7c3901cacd4ea51d6a623d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 9 Jan 2024 11:04:10 +0100 Subject: [PATCH 109/116] Add sorting to 02366_kql_summarize.sql --- .../0_stateless/02366_kql_summarize.sql | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02366_kql_summarize.sql b/tests/queries/0_stateless/02366_kql_summarize.sql index 21a1b643d98..bb12d1f251f 100644 --- a/tests/queries/0_stateless/02366_kql_summarize.sql +++ b/tests/queries/0_stateless/02366_kql_summarize.sql @@ -1,23 +1,23 @@ -- datatable(FirstName:string, LastName:string, Occupation:string, Education:string, Age:int) [ --- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28, --- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33, --- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26, --- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25, --- 'Joshua', 'Lee', 'Professional', 'Partial College', 26, --- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36, --- 'Dalton', 'Wood', 'Professional', 'Partial College', 42, --- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33, --- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28, --- 'Angel', 'Stewart', 'Professional', 'Partial College', 46, --- 'Apple', '', 'Skilled Manual', 'Bachelors', 28, +-- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28, +-- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33, +-- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26, +-- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25, +-- 'Joshua', 'Lee', 'Professional', 'Partial College', 26, +-- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36, +-- 'Dalton', 'Wood', 'Professional', 'Partial College', 42, +-- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33, +-- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28, +-- 'Angel', 'Stewart', 'Professional', 'Partial College', 46, +-- 'Apple', '', 'Skilled Manual', 'Bachelors', 28, -- dynamic(null), 'why', 'Professional', 'Partial College', 38 -- ] DROP TABLE IF EXISTS Customers; CREATE TABLE Customers -( +( FirstName Nullable(String), - LastName String, + LastName String, Occupation String, Education String, Age Nullable(UInt8) @@ -89,9 +89,9 @@ print '-- Summarize following sort --'; Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation; print '-- summarize with bin --'; -EventLog | summarize count=count() by bin(Created, 1000); -EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s); -EventLog | summarize count=count() by time_label=bin(Created/1000, 1s); +EventLog | summarize count=count() by bin(Created, 1000) | sort by count asc; +EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s) | sort by count asc; +EventLog | summarize count=count() by time_label=bin(Created/1000, 1s) | sort by count asc; Dates | project bin(datetime(EventTime), 1m); print '-- make_list_with_nulls --'; Customers | summarize t = make_list_with_nulls(FirstName); From 3875f4f4b5abfc7e2fa9f089b6bc64262d25adfc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 8 Jan 2024 23:05:49 +0100 Subject: [PATCH 110/116] Fix issue with default dataclass values --- tests/ci/ci_config.py | 58 ++++++++++++++++++++++--------------------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 895a12313da..e3319fe4a72 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,9 +1,9 @@ #!/usr/bin/env python3 -from enum import Enum import logging from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser from dataclasses import dataclass, field +from enum import Enum from pathlib import Path from typing import Callable, Dict, Iterable, List, Literal, Optional, Union @@ -47,7 +47,7 @@ class JobConfig: @num_batches - sets number of batches for multi-batch job """ - digest: DigestConfig = DigestConfig() + digest: DigestConfig = field(default_factory=DigestConfig) run_command: str = "" timeout: Optional[int] = None num_batches: int = 1 @@ -67,30 +67,32 @@ class BuildConfig: sparse_checkout: bool = False comment: str = "" static_binary_name: str = "" - job_config: JobConfig = JobConfig( - digest=DigestConfig( - include_paths=[ - "./src", - "./contrib/*-cmake", - "./contrib/consistent-hashing", - "./contrib/murmurhash", - "./contrib/libfarmhash", - "./contrib/pdqsort", - "./contrib/cityhash102", - "./contrib/sparse-checkout", - "./contrib/libmetrohash", - "./contrib/update-submodules.sh", - "./contrib/CMakeLists.txt", - "./cmake", - "./base", - "./programs", - "./packages", - "./docker/packager/packager", - ], - exclude_files=[".md"], - docker=["clickhouse/binary-builder"], - git_submodules=True, - ), + job_config: JobConfig = field( + default_factory=lambda: JobConfig( + digest=DigestConfig( + include_paths=[ + "./src", + "./contrib/*-cmake", + "./contrib/consistent-hashing", + "./contrib/murmurhash", + "./contrib/libfarmhash", + "./contrib/pdqsort", + "./contrib/cityhash102", + "./contrib/sparse-checkout", + "./contrib/libmetrohash", + "./contrib/update-submodules.sh", + "./contrib/CMakeLists.txt", + "./cmake", + "./base", + "./programs", + "./packages", + "./docker/packager/packager", + ], + exclude_files=[".md"], + docker=["clickhouse/binary-builder"], + git_submodules=True, + ), + ) ) def export_env(self, export: bool = False) -> str: @@ -107,14 +109,14 @@ class BuildConfig: @dataclass class BuildReportConfig: builds: List[str] - job_config: JobConfig = JobConfig() + job_config: JobConfig = field(default_factory=JobConfig) @dataclass class TestConfig: required_build: str force_tests: bool = False - job_config: JobConfig = JobConfig() + job_config: JobConfig = field(default_factory=JobConfig) BuildConfigs = Dict[str, BuildConfig] From 5ed4cd41d97969922423c72af6a96255ce6b862d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 9 Jan 2024 10:20:38 +0100 Subject: [PATCH 111/116] Extend S3Helper to accept a custom s3 client and endpoint --- tests/ci/s3_helper.py | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index bc403aa5015..616d645b5a6 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -5,20 +5,19 @@ import shutil import time from multiprocessing.dummy import Pool from pathlib import Path -from typing import List, Union +from typing import Any, List, Union import boto3 # type: ignore import botocore # type: ignore - -from env_helper import ( - S3_TEST_REPORTS_BUCKET, - S3_BUILDS_BUCKET, - RUNNER_TEMP, - CI, - S3_URL, - S3_DOWNLOAD, -) from compress_files import compress_file_fast +from env_helper import ( + CI, + RUNNER_TEMP, + S3_BUILDS_BUCKET, + S3_DOWNLOAD, + S3_TEST_REPORTS_BUCKET, + S3_URL, +) def _flatten_list(lst): @@ -34,11 +33,14 @@ def _flatten_list(lst): class S3Helper: max_pool_size = 100 - def __init__(self): + def __init__(self, client: Any = None, endpoint: str = S3_URL): + self.host = endpoint + if client is not None: + self.client = client + return config = botocore.config.Config(max_pool_connections=self.max_pool_size) - self.session = boto3.session.Session(region_name="us-east-1") - self.client = self.session.client("s3", endpoint_url=S3_URL, config=config) - self.host = S3_URL + session = boto3.session.Session(region_name="us-east-1") + self.client = session.client("s3", endpoint_url=endpoint, config=config) def _upload_file_to_s3( self, bucket_name: str, file_path: Path, s3_path: str @@ -199,6 +201,7 @@ class S3Helper: t = time.time() except Exception as ex: logging.critical("Failed to upload file, expcetion %s", ex) + return "" return self.s3_url(bucket_name, s3_path) p = Pool(self.max_pool_size) From 1f3ae5ec6b288378f0444cba4b647e288960216a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 9 Jan 2024 11:38:55 -0500 Subject: [PATCH 112/116] qat device init only once --- src/Compression/CompressionCodecZSTDQAT.cpp | 20 ++++++++++++++----- .../00804_test_zstd_qat_codec_compression.sql | 4 ++-- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index f7ff783bf9c..b67f1317760 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -4,7 +4,7 @@ #include #include #include - +#include #include #include @@ -23,6 +23,7 @@ class CompressionCodecZSTDQAT : public CompressionCodecZSTD public: static constexpr auto ZSTDQAT_SUPPORTED_MIN_LEVEL = 1; static constexpr auto ZSTDQAT_SUPPORTED_MAX_LEVEL = 12; + static constexpr int ZSTDQAT_DEVICE_UNINITIALIZED = 0XFFFF; explicit CompressionCodecZSTDQAT(int level_); ~CompressionCodecZSTDQAT() override; @@ -36,8 +37,11 @@ private: ZSTD_CCtx * cctx; void * sequenceProducerState; Poco::Logger * log; + static int qat_state; }; +int CompressionCodecZSTDQAT::qat_state = ZSTDQAT_DEVICE_UNINITIALIZED; + UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 source_size, char * dest) const { size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); @@ -85,9 +89,16 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) cctx = ZSTD_createCCtx(); - int res = QZSTD_startQatDevice(); + if(qat_state == ZSTDQAT_DEVICE_UNINITIALIZED) + { + qat_state = QZSTD_startQatDevice(); + if(qat_state == QZSTD_OK) + LOG_DEBUG(log, "Hardware-assisted ZSTD_QAT codec is ready!"); + else + LOG_WARNING(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, falling back to software ZSTD codec -> status: {}", qat_state); + } - if(res == QZSTD_OK) + if(qat_state == QZSTD_OK) { sequenceProducerState = QZSTD_createSeqProdState(); ZSTD_registerSequenceProducer( @@ -96,10 +107,9 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) qatSequenceProducer ); ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - LOG_DEBUG(log, "Hardware-assisted ZSTD_QAT codec is ready!"); } else - LOG_DEBUG(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, status: {} - please refer to QZSTD_Status_e in ./contrib/QAT-ZSTD-Plugin/src/qatseqprod.h", res); + sequenceProducerState = nullptr; ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); } diff --git a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql index 95943dd222c..dd4143ab1d3 100644 --- a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql @@ -4,8 +4,8 @@ SET enable_zstd_qat_codec = 1; --- Suppress test failures because stderr contains warning "Initialization of hardware-assisted DeflateQpl failed, falling --- back to software DeflateQpl coded." +-- Suppress test failures because stderr contains warning "Initialization of hardware-assisted ZSTD failed, falling +-- back to software ZSTD coded." SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS compression_codec; From e2fb33e61228493b14483bc026e9b5372a49cc7c Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 9 Jan 2024 13:42:01 -0500 Subject: [PATCH 113/116] refine zstd_qat initialization --- src/Compression/CompressionCodecZSTDQAT.cpp | 59 ++++++++------------- 1 file changed, 22 insertions(+), 37 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 02ec009b2ce..81f7995b607 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -26,7 +26,6 @@ public: static constexpr int ZSTDQAT_DEVICE_UNINITIALIZED = 0XFFFF; explicit CompressionCodecZSTDQAT(int level_); - ~CompressionCodecZSTDQAT() override; protected: bool isZstdQat() const override { return true; } @@ -34,9 +33,8 @@ protected: private: const int level; - ZSTD_CCtx * cctx = nullptr; - void * sequenceProducerState = nullptr; Poco::Logger * log; + /// Initialize QAT device and to check if it works, otherwise fallback to software ZSTD. static int qat_state; }; @@ -44,11 +42,32 @@ int CompressionCodecZSTDQAT::qat_state = ZSTDQAT_DEVICE_UNINITIALIZED; UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 source_size, char * dest) const { + ZSTD_CCtx * cctx = ZSTD_createCCtx(); + void * sequenceProducerState = nullptr; + + if(qat_state == ZSTDQAT_DEVICE_UNINITIALIZED) + { + qat_state = QZSTD_startQatDevice(); + if(qat_state == QZSTD_OK) + LOG_DEBUG(log, "Initialization of hardware-assissted ZSTD_QAT codec successful"); + else + LOG_WARNING(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, falling back to software ZSTD codec -> status: {}", qat_state); + } + if (qat_state == QZSTD_OK) + { + sequenceProducerState = QZSTD_createSeqProdState(); + ZSTD_registerSequenceProducer(cctx, sequenceProducerState, qatSequenceProducer); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + } + ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); + size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); if (ZSTD_isError(compressed_size)) throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD_QAT codec: {}", ZSTD_getErrorName(compressed_size)); + QZSTD_freeSeqProdState(sequenceProducerState); + ZSTD_freeCCtx(cctx); return static_cast(compressed_size); } @@ -86,40 +105,6 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) , log(&Poco::Logger::get("CompressionCodecZSTDQAT")) { setCodecDescription("ZSTD_QAT", {std::make_shared(static_cast(level))}); - - cctx = ZSTD_createCCtx(); - - if(qat_state == ZSTDQAT_DEVICE_UNINITIALIZED) - { - qat_state = QZSTD_startQatDevice(); - if(qat_state == QZSTD_OK) - LOG_DEBUG(log, "Initialization of hardware-assissted ZSTD_QAT codec successful"); - else - LOG_WARNING(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, falling back to software ZSTD codec -> status: {}", qat_state); - } - - if (qat_state == QZSTD_OK) - { - sequenceProducerState = QZSTD_createSeqProdState(); - ZSTD_registerSequenceProducer(cctx, sequenceProducerState, qatSequenceProducer); - ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - } - else - sequenceProducerState = nullptr; - - ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); -} - -CompressionCodecZSTDQAT::~CompressionCodecZSTDQAT() -{ - if (sequenceProducerState != nullptr) - { - QZSTD_freeSeqProdState(sequenceProducerState); - sequenceProducerState = nullptr; - } - - ZSTD_freeCCtx(cctx); - cctx = nullptr; } } From 195f10010d45d1efcf65d749cc9dc5e4e50d228d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 9 Jan 2024 10:04:12 +0000 Subject: [PATCH 114/116] Cosmetics + make things more atomic --- src/Compression/CompressionCodecZSTDQAT.cpp | 34 ++++++++++--------- .../00804_test_zstd_qat_codec_compression.sql | 3 +- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 81f7995b607..4828a71a515 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,4 +1,5 @@ #ifdef ENABLE_ZSTD_QAT_CODEC + #include #include #include @@ -34,40 +35,40 @@ protected: private: const int level; Poco::Logger * log; - /// Initialize QAT device and to check if it works, otherwise fallback to software ZSTD. - static int qat_state; + static std::atomic qat_state; /// Global initialization status of QAT device, we fall back back to software compression if uninitialized }; -int CompressionCodecZSTDQAT::qat_state = ZSTDQAT_DEVICE_UNINITIALIZED; +std::atomic CompressionCodecZSTDQAT::qat_state = ZSTDQAT_DEVICE_UNINITIALIZED; UInt32 CompressionCodecZSTDQAT::doCompressData(const char * source, UInt32 source_size, char * dest) const { - ZSTD_CCtx * cctx = ZSTD_createCCtx(); - void * sequenceProducerState = nullptr; - - if(qat_state == ZSTDQAT_DEVICE_UNINITIALIZED) + if (qat_state == ZSTDQAT_DEVICE_UNINITIALIZED) { qat_state = QZSTD_startQatDevice(); - if(qat_state == QZSTD_OK) + if (qat_state == QZSTD_OK) LOG_DEBUG(log, "Initialization of hardware-assissted ZSTD_QAT codec successful"); else LOG_WARNING(log, "Initialization of hardware-assisted ZSTD_QAT codec failed, falling back to software ZSTD codec -> status: {}", qat_state); } - if (qat_state == QZSTD_OK) - { - sequenceProducerState = QZSTD_createSeqProdState(); - ZSTD_registerSequenceProducer(cctx, sequenceProducerState, qatSequenceProducer); - ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); - } + + ZSTD_CCtx * cctx = ZSTD_createCCtx(); ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); + void * sequence_producer_state = nullptr; + if (qat_state == QZSTD_OK) + { + sequence_producer_state = QZSTD_createSeqProdState(); + ZSTD_registerSequenceProducer(cctx, sequence_producer_state, qatSequenceProducer); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_enableSeqProducerFallback, 1); + } + size_t compressed_size = ZSTD_compress2(cctx, dest, ZSTD_compressBound(source_size), source, source_size); + QZSTD_freeSeqProdState(sequence_producer_state); + ZSTD_freeCCtx(cctx); if (ZSTD_isError(compressed_size)) throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD_QAT codec: {}", ZSTD_getErrorName(compressed_size)); - QZSTD_freeSeqProdState(sequenceProducerState); - ZSTD_freeCCtx(cctx); return static_cast(compressed_size); } @@ -108,4 +109,5 @@ CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) } } + #endif diff --git a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql index dd4143ab1d3..92748efd2d1 100644 --- a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.sql @@ -4,8 +4,7 @@ SET enable_zstd_qat_codec = 1; --- Suppress test failures because stderr contains warning "Initialization of hardware-assisted ZSTD failed, falling --- back to software ZSTD coded." +-- Suppress test failures because stderr contains warning "Initialization of hardware-assisted ZSTD_QAT codec failed, falling back to software ZSTD coded." SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS compression_codec; From db3b430d4165e87d93344948af546ee8c68c7924 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 9 Jan 2024 12:53:17 +0000 Subject: [PATCH 115/116] Stylistic changes --- src/Functions/FunctionsStringDistance.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 3bb40e1c47f..6cb23bbea9f 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -297,18 +297,19 @@ struct ByteDamerauLevenshteinDistanceImpl if (haystack_size == needle_size && memcmp(haystack, needle, haystack_size) == 0) return 0; - /// Implements the algorithm for optimal string alignment distance: + /// Implements the algorithm for optimal string alignment distance from + /// https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance#Optimal_string_alignment_distance /// Dynamically allocate memory for the 2D array /// Allocating a 2D array, for convenience starts is an array of pointers to the start of the rows. - std::vector rows((needle_size + 1) * (haystack_size + 1)); + std::vector d((needle_size + 1) * (haystack_size + 1)); std::vector starts(haystack_size + 1); /// Setting the pointers in starts to the beginning of (needle_size + 1)-long intervals. /// Also initialize the row values based on the mentioned algorithm. for (size_t i = 0; i <= haystack_size; ++i) { - starts[i] = rows.data() + (needle_size + 1) * i; + starts[i] = d.data() + (needle_size + 1) * i; starts[i][0] = static_cast(i); } @@ -322,9 +323,9 @@ struct ByteDamerauLevenshteinDistanceImpl for (size_t j = 1; j <= needle_size; ++j) { int cost = (haystack[i - 1] == needle[j - 1]) ? 0 : 1; - starts[i][j] = std::min(starts[i - 1][j - 1] + cost, /// substitution - std::min(starts[i][j - 1] + 1, /// insertion - starts[i - 1][j] + 1) /// deletion + starts[i][j] = std::min(starts[i - 1][j] + 1, /// deletion + std::min(starts[i][j - 1] + 1, /// insertion + starts[i - 1][j - 1] + cost) /// substitution ); if (i > 1 && j > 1 && haystack[i - 1] == needle[j - 2] && haystack[i - 2] == needle[j - 1]) starts[i][j] = std::min(starts[i][j], starts[i - 2][j - 2] + 1); /// transposition @@ -402,6 +403,7 @@ struct ByteJaroSimilarityImpl s1i++; s2i++; } + double m = static_cast(matching_characters); double jaro_similarity = 1.0 / 3.0 * (m / static_cast(s1len) + m / static_cast(s2len) @@ -432,10 +434,7 @@ struct ByteJaroWinklerSimilarityImpl ResultType jaro_winkler_similarity = ByteJaroSimilarityImpl::process(haystack, haystack_size, needle, needle_size); - if (jaro_winkler_similarity== -1.0) - return -1.0; - - if (jaro_winkler_similarity> boost_threshold) + if (jaro_winkler_similarity > boost_threshold) { const int common_length = std::min(max_prefix_length, std::min(s1len, s2len)); int common_prefix = 0; From a80fa3f05339516d941b22d8d848362649661ccc Mon Sep 17 00:00:00 2001 From: Johnny <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 9 Jan 2024 14:28:13 -0400 Subject: [PATCH 116/116] Throws obfuscated email address elements into codeblocks. The line: > [...] tyler clickhouse com. isn't being rendered by GitHub. I'm guessing they're just escaping any tags that don't have a closing tag, or something like that. Anyway, this PR just throws those obfuscated elements into `code` tags. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c56b3c2fd0d..d356e429892 100644 --- a/README.md +++ b/README.md @@ -33,7 +33,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. +Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. ## 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"