From fb666e137c86339b532f9becc956eb4ef60971d3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 Oct 2023 23:25:20 +0000 Subject: [PATCH 001/114] 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 5b26084e4402..b581d1523695 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 5e7ece5912f3..cb8653907d33 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 41f43f4c4b42..9ea3972977fa 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -117,7 +117,7 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery 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 2db4fce81f0d..3bcd1e247970 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 c91329c869c1..1e08aec38135 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 733db0f00bc2..ae94092de4bf 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 655cb1fdb808..c970589ea33d 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 10f51563e9b8..f743f9171f85 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 ab8a37e03230..ee8f8895bed3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -229,9 +229,8 @@ class ReadFromMergeTree final : public SourceStepWithFilter 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 ae7659e074f5..a435eb498d98 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 fcf7675d15d6..eee7bf7aa1d5 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -681,7 +681,6 @@ class IStorage : public std::enable_shared_from_this, public TypePromo 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 889dcfa537f2..a4e88ebe936a 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 7e028f8c65cb..5c9226ec3706 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 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); - } - } + 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); - 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 d5d8107db485..1e57fd71bd1e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -34,7 +34,6 @@ class MergeTreeDataSelectExecutor 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 a3478069356c..92f4abdf390e 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 ec552dd10326..5a11bdf5fc0e 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 28053c84e204..e1d52eefc205 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 6d52d45c6a9c..69dbb64db38d 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 e011565edc17..b27b28e5a535 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 df03301b5e89..ee3b63f85860 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 d493fead9930..beb93afc9729 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 7354dd565522..3c68219d6d4b 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 b6253fa6daf7..ad1808d90a08 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 470e30b7947b..0f0093b6da9d 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 a4304faeaecf..2fb5bbde79bf 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -38,8 +38,6 @@ class StorageProxy : public IStorage 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 63f785a93e54..35d5925e465b 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 794991d8e062..9e342c044bd4 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -564,7 +564,6 @@ class StorageReplicatedMergeTree final : public MergeTreeData 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 @@ class StorageReplicatedMergeTree final : public MergeTreeData 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/114] 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 52c30b7b35d1..f3ff7fe06b13 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -46,7 +46,7 @@ class ShardNumColumnToFunctionVisitor : public InDepthQueryTreeVisitorWithContex 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 67187be962ca..6fe5ec2d55af 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 c95671da6be8..d0b850d987d3 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 3352567943ab..d79cf65c366b 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 0599a0fa369d..3e9d3e06493c 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 727afcb1a994..c326ff43c9e2 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 f743f9171f85..e7943fca112d 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 a24c4dbe4d08..7446203ec358 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 2606f5010099..03831ef62077 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 f5f0fa6f726b..e0a0a3313fe8 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 a4e88ebe936a..1579409e3a4a 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 ad1808d90a08..c3019b49f1fd 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 0c19634f50c2..3df4ad4a862d 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 a69f9b959551..a07479f9372e 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 69896d94bead717ebc75bdf1907935e6057ae98c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 4 Dec 2023 21:40:19 +0000 Subject: [PATCH 003/114] 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 0a8a8f1f5297..70d0e6e9b275 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 8c169dd664f3..5ec555c55963 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -690,13 +690,14 @@ class Context: public ContextData, public std::enable_shared_from_this 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 @@ class Context: public ContextData, public std::enable_shared_from_this 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 26ec8c2ce346..98fae8e362cb 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 99b4468c63c9..c063d36e288b 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 d0a97f5c74b9..38945f4945a5 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)); } -void ReadFromStorageStep::applyFilters() +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_) { - if (!context) - return; + setStepDescription(storage_name); + + for (const auto & processor : pipe.getProcessors()) + processor->setStorageLimits(query_info.storage_limits); +} +void ReadFromStorageStep::applyFilters() +{ 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 3b6ff2f06960..2eea48553b3a 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -16,32 +16,22 @@ class ReadFromPreparedSource : public SourceStepWithFilter 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 41690c1b1327..67b7e3f4a80f 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 ea10b025e878..d808966bb6a8 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 16fc0f6c09b5..ce8176c1fc10 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 1e6fbc6a75d1..2a705b801dae 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -731,6 +731,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo 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 cd4d6382bee7..9cb1fbd85061 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 03c46f8699e3..fce2d775b157 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 c3d1f39a9f6a..2acbf3f46106 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 1e7407f82c03c38caec5618058cc399afc0f3fde Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 8 Dec 2023 20:14:55 +0000 Subject: [PATCH 004/114] Customazable dependency failure handling for AsyncLoader --- src/Common/AsyncLoader.cpp | 130 ++++++++++++++---------- src/Common/AsyncLoader.h | 68 +++++++++---- src/Common/tests/gtest_async_loader.cpp | 95 +++++++++++++++++ src/Databases/DatabaseOrdinary.cpp | 10 +- 4 files changed, 228 insertions(+), 75 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 962adb8b052e..7c7b057143b3 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) + resumed_workers = job->ok(); + else if (status == LoadStatus::FAILED) + resumed_workers = job->failed(reason); + else if (status == LoadStatus::CANCELED) + resumed_workers = job->canceled(reason); + + // Adjust suspended workers count + if (resumed_workers) { - // Notify waiters - resumed_workers += job->ok(); + Pool & pool = pools[job->executionPool()]; + pool.suspended_workers -= resumed_workers; + } - // 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); - } + Info & info = scheduled_jobs[job]; + if (info.isReady()) + { + // 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; } - else + + // To avoid container modification during recursion (during clean dependency graph edges below) + LoadJobSet dependent; + dependent.swap(info.dependent_jobs); + + // Update dependent jobs + for (const auto & dpt : dependent) { - // 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); - - Info & info = scheduled_jobs[job]; - if (info.isReady()) - { - pools[job->pool_id].ready_queue.erase(info.ready_seqno); - info.ready_seqno = 0; - } + 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); - // Recurse into all dependent jobs - LoadJobSet dependent; - dependent.swap(info.dependent_jobs); // To avoid container modification during recursion - for (const auto & dep : dependent) + 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 7b6e3ebfefe7..fff36d9863aa 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 @@ class LoadJob : private boost::noncopyable 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 @@ class LoadJob : private boost::noncopyable std::atomic finish_time{TimePoint{}}; }; -struct EmptyJobFunc +// 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); + +// 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) { - void operator()(AsyncLoader &, const LoadJobPtr &) {} -}; + return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(dependency_failure), std::forward(func)); +} -template -LoadJobPtr makeLoadJob(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(std::move(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(const LoadJobSet & dependencies, 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(dependencies, std::move(name), 0, 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(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(std::move(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)); } -template -LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobFunc auto && func) { - return std::make_shared(dependencies, std::move(name), pool_id, std::forward(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 @@ class AsyncLoader : private boost::noncopyable { 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 @@ class AsyncLoader : private boost::noncopyable 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 f2e741e4b9f8..4a332079e207 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 1f344551c5ef..e3b8b03f35aa 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 005/114] 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 e3b8b03f35aa..d9e2c1f39da6 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 9b628aa9d91ed073be4bd9572d75e9cccdc03478 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 18 Dec 2023 13:25:43 +0000 Subject: [PATCH 006/114] feat: add server setting runtime reload --- .../System/StorageSystemServerSettings.cpp | 79 +++++++++++++++---- 1 file changed, 64 insertions(+), 15 deletions(-) diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 3a3acabc5a3d..ded5d8e8fae9 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,30 @@ namespace CurrentMetrics namespace DB { + +enum class RuntimeReloadType +{ + FULL, + ONLY_INCREASE, + NO, +}; + +static std::vector> getTypeEnumsAndValues() +{ + return std::vector>{ + {"Full", static_cast(RuntimeReloadType::FULL)}, + {"OnlyIncrease", static_cast(RuntimeReloadType::ONLY_INCREASE)}, + {"No", static_cast(RuntimeReloadType::NO)}, + }; +} + +struct UpdatedData { + std::string value; + RuntimeReloadType type; +}; + + + NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() { return { @@ -28,30 +53,53 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"description", std::make_shared()}, {"type", std::make_shared()}, {"is_obsolete", std::make_shared()}, - {"is_hot_reloadable", std::make_shared()} + {"is_hot_reloadable", std::make_shared()}, + {"runtime_reload", std::make_shared(getTypeEnumsAndValues())} }; } void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { // Server settings that have been reloaded from the config file. - std::unordered_map updated = { - {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, - {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, + // std::unordered_map updated = { + // {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, + // {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, + + // {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, + // {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, + + // {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, + // {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, + // {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, + + // {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, + // {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, + // {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, + // {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, + + // {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, + // {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, + // {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, + // {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} + // }; + + std::unordered_map updated = { + {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), RuntimeReloadType::FULL}}, + {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), RuntimeReloadType::FULL}}, - {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, - {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, + {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), RuntimeReloadType::FULL}}, + {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), RuntimeReloadType::FULL}}, - {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, - {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, - {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, + {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), RuntimeReloadType::FULL}}, + {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), RuntimeReloadType::FULL}}, + {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), RuntimeReloadType::FULL}}, - {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, - {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, - {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, - {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, + {"background_pool_size", {std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, + {"background_move_pool_size", {std::to_string(context->getMovesExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, + {"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, + {"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, - {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, + {"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), RuntimeReloadType::ONLY_INCREASE}}, {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} @@ -67,13 +115,14 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context const auto & it = updated.find(setting_name); res_columns[0]->insert(setting_name); - res_columns[1]->insert((it != updated.end()) ? it->second : setting.getValueString()); + res_columns[1]->insert((it != updated.end()) ? it->second.value: setting.getValueString()); res_columns[2]->insert(setting.getDefaultValueString()); res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); res_columns[6]->insert(setting.isObsolete()); res_columns[7]->insert((it != updated.end()) ? true : false); + res_columns[8]->insert((it != updated.end()) ? static_cast(it->second.type): static_cast(RuntimeReloadType::NO)); } } From 267b35ff671ec6c4c8340457ff2527829a00e52b Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 19 Dec 2023 12:24:32 +0000 Subject: [PATCH 007/114] feat: add server setting config --- src/Core/ServerSettings.cpp | 5 +- src/Core/ServerSettings.h | 185 ++++++++++-------- .../System/StorageSystemServerSettings.cpp | 61 ++---- 3 files changed, 124 insertions(+), 127 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index fbf86d3e9adb..93de0a6d5c32 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -4,10 +4,13 @@ namespace DB { -IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WITH_FLAG) void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config) { + + SERVER_SETTINGS(SET_RUNTIME_RELOAD_, "", "") + // settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h std::unordered_set settings_from_profile_allowlist = { "background_pool_size", diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index de0fff353893..a250cdd2020e 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -13,98 +13,119 @@ class AbstractConfiguration; namespace DB { -#define SERVER_SETTINGS(M, ALIAS) \ - M(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \ - M(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \ - M(UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0) \ - M(UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0) \ - M(UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0) \ - M(UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0) \ - M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \ - M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ - M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ - M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \ - M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \ - M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \ - M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ - M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ - M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ - M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \ - M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \ - M(UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0) \ - M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \ - M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \ - M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \ - M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \ - M(UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0) \ - M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \ - M(Int32, max_connections, 1024, "Max server connections.", 0) \ - M(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ - M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0) \ - M(String, default_database, "default", "Default database name.", 0) \ - M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ - M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0) \ - M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \ - M(UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0) \ - M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0) \ - M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0) \ - M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \ - M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \ - M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ - M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ - M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ +enum class RuntimeReloadType +{ + FULL, + ONLY_INCREASE, + NO, +}; + +#define SET_RUNTIME_RELOAD_(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ + this->runtime_reload_map.insert(std::make_pair(#NAME, RUNTIME_RELOAD)); + +#define M_WITH_FLAG_(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ + M(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + + +#define SERVER_SETTINGS(M_WITH_FLAG, M, ALIAS) \ + M_WITH_FLAG(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::NO) \ \ - M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ - M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ - M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ + M_WITH_FLAG(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ \ - M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \ - M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ - M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ - M(Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0) \ - M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \ - M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ - M(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \ - M(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \ - M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \ - M(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \ - M(String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0) \ - M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0) \ - M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \ - M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ + M_WITH_FLAG(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::NO) \ \ - M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ - M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ - M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \ + M_WITH_FLAG(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::NO) \ \ - M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ - M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ - M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ - M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ + M_WITH_FLAG(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::NO) \ \ - M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \ - M(Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \ - M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \ - M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \ - M(UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \ - M(UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0) \ - M(UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0) \ - M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ - M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ - M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ + M_WITH_FLAG(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::NO) \ \ - M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ - M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ - M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \ - M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ + M_WITH_FLAG(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::NO) \ + + +#define SERVER_SETTINGS_WITH_FLAG(M, ALIAS) \ + SERVER_SETTINGS(M_WITH_FLAG_, M, ALIAS) + +DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WITH_FLAG) -DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) struct ServerSettings : public BaseSettings { + std::unordered_map runtime_reload_map; + void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config); }; diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index ded5d8e8fae9..796d919253f2 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -20,13 +20,6 @@ namespace CurrentMetrics namespace DB { -enum class RuntimeReloadType -{ - FULL, - ONLY_INCREASE, - NO, -}; - static std::vector> getTypeEnumsAndValues() { return std::vector>{ @@ -61,50 +54,29 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { // Server settings that have been reloaded from the config file. - // std::unordered_map updated = { - // {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, - // {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, - - // {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, - // {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, - - // {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, - // {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, - // {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, + std::unordered_map updated = { + {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, + {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, - // {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, - // {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, - // {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, - // {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, + {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, + {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, - // {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, - // {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, - // {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, - // {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} - // }; + {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, + {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, + {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, - std::unordered_map updated = { - {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), RuntimeReloadType::FULL}}, - {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), RuntimeReloadType::FULL}}, + {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, + {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, + {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, + {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, - {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), RuntimeReloadType::FULL}}, - {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), RuntimeReloadType::FULL}}, - - {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), RuntimeReloadType::FULL}}, - {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), RuntimeReloadType::FULL}}, - {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), RuntimeReloadType::FULL}}, - - {"background_pool_size", {std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, - {"background_move_pool_size", {std::to_string(context->getMovesExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, - {"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, - {"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), RuntimeReloadType::ONLY_INCREASE}}, - - {"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), RuntimeReloadType::ONLY_INCREASE}}, + {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} }; + const auto & config = context->getConfigRef(); ServerSettings settings; settings.loadSettingsFromConfig(config); @@ -113,16 +85,17 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context { const auto & setting_name = setting.getName(); const auto & it = updated.find(setting_name); + const auto & runtime_reload_it = settings.runtime_reload_map.find(setting_name); res_columns[0]->insert(setting_name); - res_columns[1]->insert((it != updated.end()) ? it->second.value: setting.getValueString()); + res_columns[1]->insert((it != updated.end()) ? it->second: setting.getValueString()); res_columns[2]->insert(setting.getDefaultValueString()); res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); res_columns[6]->insert(setting.isObsolete()); res_columns[7]->insert((it != updated.end()) ? true : false); - res_columns[8]->insert((it != updated.end()) ? static_cast(it->second.type): static_cast(RuntimeReloadType::NO)); + res_columns[8]->insert(static_cast(runtime_reload_it != settings.runtime_reload_map.end() ? runtime_reload_it->second: RuntimeReloadType::NO)); } } From d557c0946a02f742ea478535f2d8ec3a0b7a9dbd Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 19 Dec 2023 14:12:04 +0000 Subject: [PATCH 008/114] fix server setting runtime reload type --- src/Core/ServerSettings.h | 77 ++++++++++++++++++++------------------- 1 file changed, 39 insertions(+), 38 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 39b7d523adf4..46b83bed74bb 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -17,6 +17,7 @@ enum class RuntimeReloadType { FULL, ONLY_INCREASE, + ONLY_DECREASE, NO, }; @@ -28,27 +29,27 @@ enum class RuntimeReloadType #define SERVER_SETTINGS(M_WITH_FLAG, M, ALIAS) \ - M_WITH_FLAG(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::FULL) \ M_WITH_FLAG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ M_WITH_FLAG(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::NO) \ @@ -60,17 +61,17 @@ enum class RuntimeReloadType M_WITH_FLAG(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::FULL) \ M_WITH_FLAG(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + M_WITH_FLAG(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ \ M_WITH_FLAG(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ @@ -91,27 +92,27 @@ enum class RuntimeReloadType M_WITH_FLAG(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0) \ - M_WITH_FLAG(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ - M_WITH_FLAG(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ - M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ - M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ + M_WITH_FLAG(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::ONLY_DECREASE) \ + M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::ONLY_DECREASE) \ \ - M_WITH_FLAG(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \ - M_WITH_FLAG(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \ - M_WITH_FLAG(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ + M_WITH_FLAG(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::NO, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::ONLY_INCREASE) \ + M_WITH_FLAG(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ + M_WITH_FLAG(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::NO) \ M_WITH_FLAG(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::NO) \ \ M_WITH_FLAG(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::NO) \ From 7cbb4ad7246b9796fad8c06e915fc394cea4ca65 Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 19 Dec 2023 15:20:32 +0000 Subject: [PATCH 009/114] fix macro name --- src/Core/ServerSettings.cpp | 4 +- src/Core/ServerSettings.h | 184 ++++++++++++++++++------------------ 2 files changed, 94 insertions(+), 94 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 93de0a6d5c32..77010426c4a3 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -4,12 +4,12 @@ namespace DB { -IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WITH_FLAG) +IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WRAP) void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config) { - SERVER_SETTINGS(SET_RUNTIME_RELOAD_, "", "") + SERVER_SETTINGS(SET_RUNTIME_RELOAD, "", "") // settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h std::unordered_set settings_from_profile_allowlist = { diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 46b83bed74bb..934f6c62337f 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -21,112 +21,112 @@ enum class RuntimeReloadType NO, }; -#define SET_RUNTIME_RELOAD_(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ +#define SET_RUNTIME_RELOAD(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ this->runtime_reload_map.insert(std::make_pair(#NAME, RUNTIME_RELOAD)); -#define M_WITH_FLAG_(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ +#define M_WRAP(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ M(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ -#define SERVER_SETTINGS(M_WITH_FLAG, M, ALIAS) \ - M_WITH_FLAG(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::NO) \ +#define SERVER_SETTINGS(MW, M, ALIAS) \ + MW(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::NO) \ + MW(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::NO) \ + MW(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::NO) \ + MW(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::NO) \ + MW(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::NO) \ + MW(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::NO) \ + MW(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::NO) \ + MW(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::FULL) \ + MW(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::NO) \ + MW(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ \ - M_WITH_FLAG(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::NO) \ + MW(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::NO) \ + MW(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::NO) \ + MW(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + MW(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::NO) \ + MW(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + MW(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::NO) \ + MW(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + MW(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::NO) \ + MW(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::NO) \ + MW(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::NO) \ + MW(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::NO) \ + MW(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::ONLY_DECREASE) \ - M_WITH_FLAG(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::ONLY_DECREASE) \ + MW(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::ONLY_DECREASE) \ + MW(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::ONLY_DECREASE) \ \ - M_WITH_FLAG(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::NO, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::ONLY_INCREASE) \ - M_WITH_FLAG(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ - M_WITH_FLAG(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::ONLY_INCREASE) \ + MW(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ + MW(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::NO) \ + MW(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::NO) \ \ - M_WITH_FLAG(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::NO) \ - M_WITH_FLAG(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::NO) \ + MW(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ + MW(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::NO) \ + MW(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::NO) \ -#define SERVER_SETTINGS_WITH_FLAG(M, ALIAS) \ - SERVER_SETTINGS(M_WITH_FLAG_, M, ALIAS) +#define SERVER_SETTINGS_WRAP(M, ALIAS) \ + SERVER_SETTINGS(M_WRAP, M, ALIAS) -DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WITH_FLAG) +DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WRAP) struct ServerSettings : public BaseSettings From 00576d2092577ddf4822eb076f61253012e80455 Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 19 Dec 2023 15:23:20 +0000 Subject: [PATCH 010/114] add onlyDecrease to enums --- src/Storages/System/StorageSystemServerSettings.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index a3b132f4152b..5640207f290c 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -25,6 +25,7 @@ static std::vector> getTypeEnumsAndValues() return std::vector>{ {"Full", static_cast(RuntimeReloadType::FULL)}, {"OnlyIncrease", static_cast(RuntimeReloadType::ONLY_INCREASE)}, + {"OnlyDecrease", static_cast(RuntimeReloadType::ONLY_DECREASE)}, {"No", static_cast(RuntimeReloadType::NO)}, }; } From 3d8933f2410687d43d8e5d5b8d538b4c79c4f740 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 19 Dec 2023 10:44:44 -0500 Subject: [PATCH 011/114] Revert "Revert "Add system.dropped_tables_parts table"" --- .../system-tables/dropped_tables_parts.md | 14 +++ .../StorageSystemDroppedTablesParts.cpp | 89 +++++++++++++++++++ .../System/StorageSystemDroppedTablesParts.h | 33 +++++++ src/Storages/System/StorageSystemParts.h | 2 +- .../System/StorageSystemPartsBase.cpp | 68 +++----------- src/Storages/System/StorageSystemPartsBase.h | 81 +++++++++++++++-- src/Storages/System/attachSystemTables.cpp | 2 + 7 files changed, 225 insertions(+), 64 deletions(-) create mode 100644 docs/en/operations/system-tables/dropped_tables_parts.md create mode 100644 src/Storages/System/StorageSystemDroppedTablesParts.cpp create mode 100644 src/Storages/System/StorageSystemDroppedTablesParts.h diff --git a/docs/en/operations/system-tables/dropped_tables_parts.md b/docs/en/operations/system-tables/dropped_tables_parts.md new file mode 100644 index 000000000000..095f35287fe9 --- /dev/null +++ b/docs/en/operations/system-tables/dropped_tables_parts.md @@ -0,0 +1,14 @@ +--- +slug: /en/operations/system-tables/dropped_tables_parts +--- +# dropped_tables_parts {#system_tables-dropped_tables_parts} + +Contains information about parts of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) dropped tables from [system.dropped_tables](./dropped_tables.md) + +The schema of this table is the same as [system.parts](./parts.md) + +**See Also** + +- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) +- [system.parts](./parts.md) +- [system.dropped_tables](./dropped_tables.md) diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp new file mode 100644 index 000000000000..9253cc99d721 --- /dev/null +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -0,0 +1,89 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +StoragesDroppedInfoStream::StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context) + : StoragesInfoStreamBase(context) +{ + needsLock = false; + + /// Will apply WHERE to subset of columns and then add more columns. + /// This is kind of complicated, but we use WHERE to do less work. + + Block block_to_filter; + + MutableColumnPtr database_column_mut = ColumnString::create(); + MutableColumnPtr table_column_mut = ColumnString::create(); + MutableColumnPtr engine_column_mut = ColumnString::create(); + MutableColumnPtr active_column_mut = ColumnUInt8::create(); + MutableColumnPtr storage_uuid_column_mut = ColumnUUID::create(); + + const auto access = context->getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); + + auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); + for (const auto & dropped_table : tables_mark_dropped) + { + StoragePtr storage = dropped_table.table; + if (!storage) + continue; + + UUID storage_uuid = storage->getStorageID().uuid; + String database_name = storage->getStorageID().getDatabaseName(); + String table_name = storage->getStorageID().getTableName(); + String engine_name = storage->getName(); +#if USE_MYSQL + if (auto * proxy = dynamic_cast(storage.get())) + { + auto nested = proxy->getNested(); + storage.swap(nested); + } +#endif + if (!dynamic_cast(storage.get())) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + storages[storage_uuid] = storage; + + /// Add all combinations of flag 'active'. + for (UInt64 active : {0, 1}) + { + database_column_mut->insert(database_name); + table_column_mut->insert(table_name); + engine_column_mut->insert(engine_name); + active_column_mut->insert(active); + storage_uuid_column_mut->insert(storage_uuid); + } + } + + block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared(), "database")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared(), "table")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared(), "engine")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared(), "active")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), "uuid")); + + if (block_to_filter.rows()) + { + /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + rows = block_to_filter.rows(); + } + + database_column = block_to_filter.getByName("database").column; + table_column = block_to_filter.getByName("table").column; + active_column = block_to_filter.getByName("active").column; + storage_uuid_column = block_to_filter.getByName("uuid").column; +} + + +} diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.h b/src/Storages/System/StorageSystemDroppedTablesParts.h new file mode 100644 index 000000000000..1a8a27e0b7c7 --- /dev/null +++ b/src/Storages/System/StorageSystemDroppedTablesParts.h @@ -0,0 +1,33 @@ +#pragma once + +#include + + +namespace DB +{ + +class StoragesDroppedInfoStream : public StoragesInfoStreamBase +{ +public: + StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context); +}; + +class Context; + + +/** Implements system table 'dropped_tables_parts' which allows to get information about data parts for dropped but not yet removed tables. + */ +class StorageSystemDroppedTablesParts final : public StorageSystemParts +{ +public: + explicit StorageSystemDroppedTablesParts(const StorageID & table_id) : StorageSystemParts(table_id) {} + + std::string getName() const override { return "SystemDroppedTablesParts"; } +protected: + std::unique_ptr getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) override + { + return std::make_unique(query_info, context); + } +}; + +} diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h index c7a46cfda54d..e0082e40e7d5 100644 --- a/src/Storages/System/StorageSystemParts.h +++ b/src/Storages/System/StorageSystemParts.h @@ -11,7 +11,7 @@ class Context; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. */ -class StorageSystemParts final : public StorageSystemPartsBase +class StorageSystemParts : public StorageSystemPartsBase { public: explicit StorageSystemParts(const StorageID & table_id_); diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 8d2e29007223..7b2e9fe18a7a 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -22,11 +23,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - bool StorageSystemPartsBase::hasStateColumn(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) { bool has_state_column = false; @@ -83,7 +79,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) - : query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()) + : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. /// This is kind of complicated, but we use WHERE to do less work. @@ -93,6 +89,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte MutableColumnPtr table_column_mut = ColumnString::create(); MutableColumnPtr engine_column_mut = ColumnString::create(); MutableColumnPtr active_column_mut = ColumnUInt8::create(); + MutableColumnPtr storage_uuid_column_mut = ColumnUUID::create(); const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); @@ -139,6 +136,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte continue; String engine_name = storage->getName(); + UUID storage_uuid = storage->getStorageID().uuid; #if USE_MYSQL if (auto * proxy = dynamic_cast(storage.get())) @@ -153,7 +151,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; - storages[std::make_pair(database_name, iterator->name())] = storage; + storages[storage_uuid] = storage; /// Add all combinations of flag 'active'. for (UInt64 active : {0, 1}) @@ -161,6 +159,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte table_column_mut->insert(table_name); engine_column_mut->insert(engine_name); active_column_mut->insert(active); + storage_uuid_column_mut->insert(storage_uuid); } offsets[i] += 2; @@ -178,6 +177,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared(), "table")); block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared(), "engine")); block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared(), "active")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), "uuid")); if (rows) { @@ -189,57 +189,9 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte database_column = block_to_filter.getByName("database").column; table_column = block_to_filter.getByName("table").column; active_column = block_to_filter.getByName("active").column; - - next_row = 0; + storage_uuid_column = block_to_filter.getByName("uuid").column; } -StoragesInfo StoragesInfoStream::next() -{ - while (next_row < rows) - { - StoragesInfo info; - - info.database = (*database_column)[next_row].get(); - info.table = (*table_column)[next_row].get(); - - auto is_same_table = [&info, this] (size_t row) -> bool - { - return (*database_column)[row].get() == info.database && - (*table_column)[row].get() == info.table; - }; - - /// We may have two rows per table which differ in 'active' value. - /// If rows with 'active = 0' were not filtered out, this means we - /// must collect the inactive parts. Remember this fact in StoragesInfo. - for (; next_row < rows && is_same_table(next_row); ++next_row) - { - const auto active = (*active_column)[next_row].get(); - if (active == 0) - info.need_inactive_parts = true; - } - - info.storage = storages.at(std::make_pair(info.database, info.table)); - - /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout); - - if (info.table_lock == nullptr) - { - // Table was dropped while acquiring the lock, skipping table - continue; - } - - info.engine = info.storage->getName(); - - info.data = dynamic_cast(info.storage.get()); - if (!info.data) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown engine {}", info.engine); - - return info; - } - - return {}; -} Pipe StorageSystemPartsBase::read( const Names & column_names, @@ -252,7 +204,7 @@ Pipe StorageSystemPartsBase::read( { bool has_state_column = hasStateColumn(column_names, storage_snapshot); - StoragesInfoStream stream(query_info, context); + auto stream = getStoragesInfoStream(query_info, context); /// Create the result. Block sample = storage_snapshot->metadata->getSampleBlock(); @@ -263,7 +215,7 @@ Pipe StorageSystemPartsBase::read( if (has_state_column) res_columns.push_back(ColumnString::create()); - while (StoragesInfo info = stream.next()) + while (StoragesInfo info = stream->next()) { processNextStorage(context, res_columns, columns_mask, info, has_state_column); } diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index c3d2e64b303d..fd20b0756b25 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class Context; struct StoragesInfo @@ -29,13 +34,64 @@ struct StoragesInfo }; /** A helper class that enumerates the storages that match given query. */ -class StoragesInfoStream +class StoragesInfoStreamBase { public: - StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context); - StoragesInfo next(); + StoragesInfoStreamBase(ContextPtr context) + : query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()), next_row(0), rows(0) + {} + + StoragesInfo next() + { + while (next_row < rows) + { + StoragesInfo info; + + info.database = (*database_column)[next_row].get(); + info.table = (*table_column)[next_row].get(); + UUID storage_uuid = (*storage_uuid_column)[next_row].get(); + + auto is_same_table = [&storage_uuid, this] (size_t row) -> bool + { + return (*storage_uuid_column)[row].get() == storage_uuid; + }; + + /// We may have two rows per table which differ in 'active' value. + /// If rows with 'active = 0' were not filtered out, this means we + /// must collect the inactive parts. Remember this fact in StoragesInfo. + for (; next_row < rows && is_same_table(next_row); ++next_row) + { + const auto active = (*active_column)[next_row].get(); + if (active == 0) + info.need_inactive_parts = true; + } + + info.storage = storages.at(storage_uuid); + + if (needsLock) + { + /// For table not to be dropped and set of columns to remain constant. + info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout); + if (info.table_lock == nullptr) + { + // Table was dropped while acquiring the lock, skipping table + continue; + } + } + + info.engine = info.storage->getName(); + + info.data = dynamic_cast(info.storage.get()); + if (!info.data) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown engine {}", info.engine); + + return info; + } + + return {}; + } -private: +protected: String query_id; Settings settings; @@ -43,12 +99,22 @@ class StoragesInfoStream ColumnPtr database_column; ColumnPtr table_column; ColumnPtr active_column; + ColumnPtr storage_uuid_column; size_t next_row; size_t rows; - using StoragesMap = std::map, StoragePtr>; + using StoragesMap = std::unordered_map; StoragesMap storages; + + bool needsLock = true; +}; + + +class StoragesInfoStream : public StoragesInfoStreamBase +{ +public: + StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context); }; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. @@ -77,6 +143,11 @@ class StorageSystemPartsBase : public IStorage StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_); + virtual std::unique_ptr getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) + { + return std::make_unique(query_info, context); + } + virtual void processNextStorage(ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0; }; diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index ffe74f1c94ae..57f37c8899eb 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -82,6 +82,7 @@ #include #include #include +#include #include #include #include @@ -156,6 +157,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "backups"); attach(context, system_database, "schema_inference_cache"); attach(context, system_database, "dropped_tables"); + attach(context, system_database, "dropped_tables_parts"); attach(context, system_database, "scheduler"); #if defined(__ELF__) && !defined(OS_FREEBSD) attach(context, system_database, "symbols"); From 17c3212419566ae5e2dc8ad2b4cd460e7f0b81de Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 19 Dec 2023 10:51:43 -0500 Subject: [PATCH 012/114] Update 02698_marked_dropped_tables.sql --- tests/queries/0_stateless/02698_marked_dropped_tables.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02698_marked_dropped_tables.sql b/tests/queries/0_stateless/02698_marked_dropped_tables.sql index 9bf6579b5836..a3686159a780 100644 --- a/tests/queries/0_stateless/02698_marked_dropped_tables.sql +++ b/tests/queries/0_stateless/02698_marked_dropped_tables.sql @@ -4,8 +4,11 @@ SET database_atomic_wait_for_drop_and_detach_synchronously = 0; DROP TABLE IF EXISTS 25400_dropped_tables; CREATE TABLE 25400_dropped_tables (id Int32) Engine=MergeTree() ORDER BY id; +INSERT INTO 25400_dropped_tables VALUES (1),(2); +INSERT INTO 25400_dropped_tables VALUES (3),(4); DROP TABLE 25400_dropped_tables; SELECT table, engine FROM system.dropped_tables WHERE database = currentDatabase() LIMIT 1; DESCRIBE TABLE system.dropped_tables; - +SELECT database, table, name FROM system.dropped_tables_parts WHERE database = currentDatabase() and table = '25400_dropped_tables'; +DESCRIBE TABLE system.dropped_tables_parts; From f0a20c6c321a98027605d53fe48c145ea5c6fbd6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 19 Dec 2023 10:52:33 -0500 Subject: [PATCH 013/114] Update 02698_marked_dropped_tables.reference --- .../02698_marked_dropped_tables.reference | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/tests/queries/0_stateless/02698_marked_dropped_tables.reference b/tests/queries/0_stateless/02698_marked_dropped_tables.reference index 44906da9527c..84e478cc3832 100644 --- a/tests/queries/0_stateless/02698_marked_dropped_tables.reference +++ b/tests/queries/0_stateless/02698_marked_dropped_tables.reference @@ -6,3 +6,71 @@ uuid UUID engine String metadata_dropped_path String table_dropped_time DateTime +default 25400_dropped_tables all_1_1_0 +default 25400_dropped_tables all_2_2_0 +partition String +name String +uuid UUID +part_type String +active UInt8 +marks UInt64 +rows UInt64 +bytes_on_disk UInt64 +data_compressed_bytes UInt64 +data_uncompressed_bytes UInt64 +primary_key_size UInt64 +marks_bytes UInt64 +secondary_indices_compressed_bytes UInt64 +secondary_indices_uncompressed_bytes UInt64 +secondary_indices_marks_bytes UInt64 +modification_time DateTime +remove_time DateTime +refcount UInt32 +min_date Date +max_date Date +min_time DateTime +max_time DateTime +partition_id String +min_block_number Int64 +max_block_number Int64 +level UInt32 +data_version UInt64 +primary_key_bytes_in_memory UInt64 +primary_key_bytes_in_memory_allocated UInt64 +is_frozen UInt8 +database String +table String +engine String +disk_name String +path String +hash_of_all_files String +hash_of_uncompressed_files String +uncompressed_hash_of_compressed_files String +delete_ttl_info_min DateTime +delete_ttl_info_max DateTime +move_ttl_info.expression Array(String) +move_ttl_info.min Array(DateTime) +move_ttl_info.max Array(DateTime) +default_compression_codec String +recompression_ttl_info.expression Array(String) +recompression_ttl_info.min Array(DateTime) +recompression_ttl_info.max Array(DateTime) +group_by_ttl_info.expression Array(String) +group_by_ttl_info.min Array(DateTime) +group_by_ttl_info.max Array(DateTime) +rows_where_ttl_info.expression Array(String) +rows_where_ttl_info.min Array(DateTime) +rows_where_ttl_info.max Array(DateTime) +projections Array(String) +visible UInt8 +creation_tid Tuple(UInt64, UInt64, UUID) +removal_tid_lock UInt64 +removal_tid Tuple(UInt64, UInt64, UUID) +creation_csn UInt64 +removal_csn UInt64 +has_lightweight_delete UInt8 +last_removal_attempt_time DateTime +removal_state String +bytes UInt64 ALIAS bytes_on_disk +marks_size UInt64 ALIAS marks_bytes +part_name String ALIAS name From 342a4f7b96cce6d8375dfd5ac95b5581c5633afb Mon Sep 17 00:00:00 2001 From: skyoct Date: Thu, 21 Dec 2023 14:25:18 +0000 Subject: [PATCH 014/114] add doc and opt some code --- .../system-tables/server_settings.md | 37 ++-- src/Core/ServerSettings.cpp | 1 - src/Core/ServerSettings.h | 190 +++++++++--------- .../System/StorageSystemServerSettings.cpp | 28 +-- 4 files changed, 125 insertions(+), 131 deletions(-) diff --git a/docs/en/operations/system-tables/server_settings.md b/docs/en/operations/system-tables/server_settings.md index 7efe605ccef6..6374b2d02a2b 100644 --- a/docs/en/operations/system-tables/server_settings.md +++ b/docs/en/operations/system-tables/server_settings.md @@ -14,6 +14,12 @@ Columns: - `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml` - `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description. - `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type. +- `is_hot_reloadable` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether hot reload is supported +- `runtime_reload` ([Enum8](../../sql-reference/data-types/enum.md)) — Hot reload type of parameter. Possible values: + - `Yes` + - `OnlyIncrease` + - `OnlyDecrease` + - `No` - `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete. **Example** @@ -27,22 +33,21 @@ WHERE name LIKE '%thread_pool%' ``` ``` text -┌─name────────────────────────────────────────_─value─_─default─_─changed─_─description────────────────────────────────────────────────────────────────────────────────────────────────────── -───────────────────────────────────_─type───_─is_obsolete─┐ -│ max_thread_pool_size │ 10000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │ -│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │ -│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │ -│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │ -│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │ -│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │ -│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │ -│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │ -│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │ -│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │ -│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │ -│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │ -└─────────────────────────────────────────────┴───────┴─────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────── -───────────────────────────────────┴────────┴─────────────┘ +┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─is_hot_reloadable─┬─runtime_reload─┬─is_obsolete─┐ +│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │ No │ 0 │ +│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │ No │ 0 │ +│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │ No │ 0 │ +│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │ Yes │ 0 │ +│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ +│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ +│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │ Yes │ 0 │ +│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │ Yes │ 0 │ +│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │ Yes │ 0 │ +│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │ Yes │ 0 │ +│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ +│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ +└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴───────────────────┴────────────────┴─────────────┘ + ``` Using of `WHERE changed` can be useful, for example, when you want to check diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 77010426c4a3..7f48226c2139 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -10,7 +10,6 @@ void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfigurat { SERVER_SETTINGS(SET_RUNTIME_RELOAD, "", "") - // settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h std::unordered_set settings_from_profile_allowlist = { "background_pool_size", diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 10fac354b5cd..3215a35b4831 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -9,16 +9,15 @@ namespace Poco::Util { class AbstractConfiguration; } - namespace DB { enum class RuntimeReloadType { - FULL, - ONLY_INCREASE, - ONLY_DECREASE, - NO, + Yes, + OnlyIncrease, + OnlyDecrease, + No, }; #define SET_RUNTIME_RELOAD(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ @@ -29,102 +28,102 @@ enum class RuntimeReloadType #define SERVER_SETTINGS(MW, M, ALIAS) \ - MW(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::NO) \ - MW(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::NO) \ - MW(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::NO) \ - MW(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::NO) \ - MW(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::NO) \ - MW(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::NO) \ - MW(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::NO) \ - MW(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - MW(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - MW(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::FULL) \ - MW(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::NO) \ - MW(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::NO) \ + MW(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::No) \ + MW(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::No) \ + MW(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::No) \ + MW(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::No) \ + MW(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::No) \ + MW(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::No) \ + MW(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::No) \ + MW(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ + MW(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ + MW(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::Yes) \ + MW(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::No) \ + MW(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::No) \ \ - MW(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::FULL) \ + MW(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ \ - MW(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::NO) \ - MW(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::NO) \ - MW(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - MW(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::NO) \ - MW(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - MW(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::NO) \ - MW(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - MW(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::NO) \ - MW(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::NO) \ + MW(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::No) \ + MW(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::No) \ + MW(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ + MW(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::No) \ + MW(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ + MW(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::No) \ + MW(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ + MW(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::No) \ + MW(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::No) \ \ - MW(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::NO) \ - MW(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::NO) \ - MW(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::NO) \ + MW(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::No) \ + MW(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::No) \ + MW(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::No) \ \ - MW(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::ONLY_DECREASE) \ - MW(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::ONLY_DECREASE) \ + MW(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::OnlyDecrease) \ + MW(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::OnlyDecrease) \ \ - MW(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::ONLY_INCREASE) \ - MW(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ - MW(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::FULL) \ - MW(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::NO) \ - MW(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::NO) \ + MW(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::OnlyIncrease) \ + MW(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::Yes) \ + MW(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::Yes) \ + MW(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::No) \ + MW(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::No) \ \ - MW(M, Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0, RuntimeReloadType::NO) \ - MW(M, Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0, RuntimeReloadType::NO) \ - MW(M, Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0, RuntimeReloadType::NO) \ - MW(M, Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::NO) \ - MW(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ - MW(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::NO) \ - MW(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::NO) \ - MW(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::NO) \ + MW(M, Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0, RuntimeReloadType::No) \ + MW(M, Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ + MW(M, Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ + MW(M, Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::No) \ + MW(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::No) \ + MW(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::No) \ + MW(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::No) \ + MW(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::No) \ #define SERVER_SETTINGS_WRAP(M, ALIAS) \ @@ -136,7 +135,6 @@ DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WRAP) struct ServerSettings : public BaseSettings { std::unordered_map runtime_reload_map; - void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config); }; diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 5640207f290c..bcfca59618a3 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -20,23 +20,16 @@ namespace CurrentMetrics namespace DB { -static std::vector> getTypeEnumsAndValues() +static std::vector> getRuntimeReloadEnumAndValues() { return std::vector>{ - {"Full", static_cast(RuntimeReloadType::FULL)}, - {"OnlyIncrease", static_cast(RuntimeReloadType::ONLY_INCREASE)}, - {"OnlyDecrease", static_cast(RuntimeReloadType::ONLY_DECREASE)}, - {"No", static_cast(RuntimeReloadType::NO)}, + {"Yes", static_cast(RuntimeReloadType::Yes)}, + {"OnlyIncrease", static_cast(RuntimeReloadType::OnlyIncrease)}, + {"OnlyDecrease", static_cast(RuntimeReloadType::OnlyDecrease)}, + {"No", static_cast(RuntimeReloadType::No)}, }; } -struct UpdatedData { - std::string value; - RuntimeReloadType type; -}; - - - NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() { return { @@ -46,9 +39,9 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, - {"is_obsolete", std::make_shared()}, {"is_hot_reloadable", std::make_shared()}, - {"runtime_reload", std::make_shared(getTypeEnumsAndValues())} + {"runtime_reload", std::make_shared(getRuntimeReloadEnumAndValues())}, + {"is_obsolete", std::make_shared()} }; } @@ -88,7 +81,6 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context { const auto & setting_name = setting.getName(); const auto & it = updated.find(setting_name); - const auto & runtime_reload_it = settings.runtime_reload_map.find(setting_name); res_columns[0]->insert(setting_name); res_columns[1]->insert((it != updated.end()) ? it->second: setting.getValueString()); @@ -96,9 +88,9 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); - res_columns[6]->insert(setting.isObsolete()); - res_columns[7]->insert((it != updated.end()) ? true : false); - res_columns[8]->insert(static_cast(runtime_reload_it != settings.runtime_reload_map.end() ? runtime_reload_it->second: RuntimeReloadType::NO)); + res_columns[6]->insert((it != updated.end()) ? true : false); + res_columns[7]->insert(static_cast(settings.runtime_reload_map.contains(setting_name) ? settings.runtime_reload_map.at(setting_name): RuntimeReloadType::No)); + res_columns[8]->insert(setting.isObsolete()); } } From 2ad9c7d96d52d2a5b84b2038bd38ebb12a2379ad Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Dec 2023 16:56:02 +0000 Subject: [PATCH 015/114] fix for Ordinary --- src/Storages/System/StorageSystemPartsBase.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 7b2e9fe18a7a..0e7c33f2c24a 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -1,3 +1,4 @@ +#include "Common/SipHash.h" #include #include #include @@ -137,6 +138,13 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte String engine_name = storage->getName(); UUID storage_uuid = storage->getStorageID().uuid; + if (database->getEngineName() == "Ordinary") + { + SipHash hash; + hash.update(database_name); + hash.update(table_name); + storage_uuid = hash.get128(); + } #if USE_MYSQL if (auto * proxy = dynamic_cast(storage.get())) From d7fe86279f54cffe6a19b31584b705922d49c6ba Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 23 Dec 2023 03:47:43 +0000 Subject: [PATCH 016/114] StoragesInfoStreamBase refactoring, additional test, style fix --- .../StorageSystemDroppedTablesParts.cpp | 2 -- .../System/StorageSystemDroppedTablesParts.h | 6 +++++ .../System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 26 +++++++++---------- .../02947_dropped_tables_parts.reference | 2 ++ .../02947_dropped_tables_parts.sql | 13 ++++++++++ 6 files changed, 35 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02947_dropped_tables_parts.reference create mode 100644 tests/queries/0_stateless/02947_dropped_tables_parts.sql diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index 9253cc99d721..bbe99c688c98 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -13,8 +13,6 @@ namespace DB StoragesDroppedInfoStream::StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context) : StoragesInfoStreamBase(context) { - needsLock = false; - /// Will apply WHERE to subset of columns and then add more columns. /// This is kind of complicated, but we use WHERE to do less work. diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.h b/src/Storages/System/StorageSystemDroppedTablesParts.h index 1a8a27e0b7c7..f548697a6a95 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.h +++ b/src/Storages/System/StorageSystemDroppedTablesParts.h @@ -10,6 +10,12 @@ class StoragesDroppedInfoStream : public StoragesInfoStreamBase { public: StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context); +protected: + bool tryLockTable(StoragesInfo &) override + { + // we don't need to lock a dropped table + return true; + } }; class Context; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 0e7c33f2c24a..a5eb475ef744 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -1,4 +1,4 @@ -#include "Common/SipHash.h" +#include #include #include #include diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index fd20b0756b25..ce7994e7a9db 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -41,6 +41,9 @@ class StoragesInfoStreamBase : query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()), next_row(0), rows(0) {} + StoragesInfoStreamBase(const StoragesInfoStreamBase&) = default; + virtual ~StoragesInfoStreamBase() = default; + StoragesInfo next() { while (next_row < rows) @@ -68,16 +71,9 @@ class StoragesInfoStreamBase info.storage = storages.at(storage_uuid); - if (needsLock) - { - /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout); - if (info.table_lock == nullptr) - { - // Table was dropped while acquiring the lock, skipping table - continue; - } - } + /// For table not to be dropped and set of columns to remain constant. + if (!tryLockTable(info)) + continue; info.engine = info.storage->getName(); @@ -90,7 +86,13 @@ class StoragesInfoStreamBase return {}; } - +protected: + virtual bool tryLockTable(StoragesInfo & info) + { + info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout); + // nullptr means table was dropped while acquiring the lock + return info.table_lock != nullptr; + } protected: String query_id; Settings settings; @@ -106,8 +108,6 @@ class StoragesInfoStreamBase using StoragesMap = std::unordered_map; StoragesMap storages; - - bool needsLock = true; }; diff --git a/tests/queries/0_stateless/02947_dropped_tables_parts.reference b/tests/queries/0_stateless/02947_dropped_tables_parts.reference new file mode 100644 index 000000000000..086d55c3d93a --- /dev/null +++ b/tests/queries/0_stateless/02947_dropped_tables_parts.reference @@ -0,0 +1,2 @@ +default 02947_table_1 all_1_1_0 +default 02947_table_2 all_1_1_0 diff --git a/tests/queries/0_stateless/02947_dropped_tables_parts.sql b/tests/queries/0_stateless/02947_dropped_tables_parts.sql new file mode 100644 index 000000000000..265862b448c2 --- /dev/null +++ b/tests/queries/0_stateless/02947_dropped_tables_parts.sql @@ -0,0 +1,13 @@ + +DROP TABLE IF EXISTS 02947_table_1; +DROP TABLE IF EXISTS 02947_table_2; + +CREATE TABLE 02947_table_1 (id Int32) Engine=MergeTree() ORDER BY id; +CREATE TABLE 02947_table_2 (id Int32) Engine=MergeTree() ORDER BY id; +INSERT INTO 02947_table_1 VALUES (1),(2); +INSERT INTO 02947_table_2 VALUES (3),(4); + +SELECT database, table, name FROM system.parts WHERE database = currentDatabase() AND startsWith(table, '02947_table_'); + +DROP TABLE 02947_table_1; +DROP TABLE 02947_table_2; From 62139f0073b0194294293b3c9a9a463525139794 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 29 Dec 2023 12:19:56 +0000 Subject: [PATCH 017/114] fix recursion for finish --- src/Common/AsyncLoader.cpp | 37 ++++++++++++++++++++++--------------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 7c7b057143b3..33dc0d8cf33f 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 (auto dpt_info = scheduled_jobs.find(dpt); dpt_info != scheduled_jobs.end()) + { + dpt_info->second.dependencies_left--; + if (!dpt_info->second.isBlocked()) + enqueue(dpt_info->second, dpt, lock); - if (status != LoadStatus::OK) + 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 { - 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); + // 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 018/114] 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 8de6a5e2615a..85de309b1533 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 019/114] 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 33dc0d8cf33f..0e0fa25e7a12 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 020/114] 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 75b2a852c1c7..63dc4f528b0a 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 a407cd270d97b2bb06762da387410c0dec651a11 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Tue, 21 Nov 2023 09:20:21 -0800 Subject: [PATCH 021/114] Added new library stl-cpp for sesonal decompose of time series data --- contrib/CMakeLists.txt | 1 + contrib/stl-cpp | 1 + contrib/stl-cpp-cmake/CMakeLists.txt | 10 ++++++++++ src/CMakeLists.txt | 4 ++++ src/Common/config.h.in | 1 + src/Functions/CMakeLists.txt | 4 ++++ src/configure_config.cmake | 3 +++ utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 8 files changed, 25 insertions(+) create mode 160000 contrib/stl-cpp create mode 100644 contrib/stl-cpp-cmake/CMakeLists.txt diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1b5ba15187f8..84e3e8c0c7e4 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -77,6 +77,7 @@ add_contrib (minizip-ng-cmake minizip-ng) add_contrib (snappy-cmake snappy) add_contrib (rocksdb-cmake rocksdb) add_contrib (thrift-cmake thrift) +add_contrib (stl-cpp-cmake stl-cpp) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion add_contrib (avro-cmake avro) # requires: snappy diff --git a/contrib/stl-cpp b/contrib/stl-cpp new file mode 160000 index 000000000000..f3e1cd02b4f0 --- /dev/null +++ b/contrib/stl-cpp @@ -0,0 +1 @@ +Subproject commit f3e1cd02b4f0341a7539ed1a171b21b14b37ed51 diff --git a/contrib/stl-cpp-cmake/CMakeLists.txt b/contrib/stl-cpp-cmake/CMakeLists.txt new file mode 100644 index 000000000000..cd04783beb19 --- /dev/null +++ b/contrib/stl-cpp-cmake/CMakeLists.txt @@ -0,0 +1,10 @@ +option (ENABLE_SEASONAL "Enable stl-cpp" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_SEASONAL) + message(STATUS "Not using stl-cpp") + return() +endif() +#set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/stl-cpp") +add_library(_stl-cpp INTERFACE) +target_include_directories(_stl-cpp INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/stl-cpp/include) +add_library(ch_contrib::stl-cpp ALIAS _stl-cpp) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 86cb9acd0565..b04b1575b18e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -396,6 +396,10 @@ if (TARGET ch_contrib::nuraft) target_link_libraries (clickhouse_compression PUBLIC ch_contrib::nuraft) endif() +if (TARGET ch_contrib::stl-cpp) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::stl-cpp) +endif () + dbms_target_link_libraries ( PRIVATE boost::filesystem diff --git a/src/Common/config.h.in b/src/Common/config.h.in index f84e28942c54..8f04f7a1eb8f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,6 +59,7 @@ #cmakedefine01 USE_PRQL #cmakedefine01 USE_OPENSSL_INTREE #cmakedefine01 USE_ULID +#cmakedefine01 USE_SEASONAL #cmakedefine01 FIU_ENABLE #cmakedefine01 USE_BCRYPT #cmakedefine01 USE_LIBARCHIVE diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 896765945817..8133356f8288 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -75,6 +75,10 @@ if (TARGET ch_contrib::base64) list (APPEND PRIVATE_LIBS ch_contrib::base64) endif() +if (TARGET ch_contrib::stl-cpp) + list (APPEND PRIVATE_LIBS ch_contrib::stl-cpp) +endif() + if (ENABLE_NLP) list (APPEND PRIVATE_LIBS ch_contrib::cld2) endif() diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 9358abdf7f8a..c003ec466b77 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -161,6 +161,9 @@ endif () if (ENABLE_OPENSSL) set(USE_OPENSSL_INTREE 1) endif () +if (TARGET ch_contrib::stl-cpp) + set(USE_SEASONAL 1) +endif() if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d68330771e5b..c39143b40e0d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2239,6 +2239,7 @@ sequenceCount sequenceMatch sequenceNextNode seriesPeriodDetectFFT +seriesDecomposeSTL serverTimeZone serverTimezone serverUUID From 5c34ddab094b0a1312e167e687e35fa0a62ae5a8 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Tue, 21 Nov 2023 12:25:21 -0800 Subject: [PATCH 022/114] implemneted seriesDecomposeSTL method --- .../functions/time-series-functions.md | 54 ++++- src/Functions/seriesDecomposeSTL.cpp | 184 ++++++++++++++++++ .../02813_series_decompose.reference | 2 + .../0_stateless/02813_series_decompose.sql | 10 + 4 files changed, 249 insertions(+), 1 deletion(-) create mode 100644 src/Functions/seriesDecomposeSTL.cpp create mode 100644 tests/queries/0_stateless/02813_series_decompose.reference create mode 100644 tests/queries/0_stateless/02813_series_decompose.sql diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 434432baa483..627744836a0d 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -4,7 +4,7 @@ sidebar_position: 172 sidebar_label: Time Series --- -# Time Series Functions +#Time Series Functions Below functions are used for time series analysis. @@ -57,3 +57,55 @@ Result: │ 6 │ └─────────┘ ``` + +## seriesDecomposeSTL + +Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) +Returns an array of arrays where the first array include seasonal components, the second array - trend, +and the third array - residue component. + +**Syntax** + +``` sql +seriesDecomposeSTL(series, period); +``` + +**Arguments** + +- `series` - An array of numeric values +- `period` - A positive number + +**Returned value** + +- Array of arrays + +Type: [Array](../../sql-reference/data-types/array.md). + +**Examples** + +Query: + +``` sql +SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3) AS print_0; +``` + +Result: + +``` text +┌───────────print_0──────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ [[ + -13.529999, -3.1799996, 16.71, -13.53, -3.1799996, 16.71, -13.53, -3.1799996, + 16.71, -13.530001, -3.18, 16.710001, -13.530001, -3.1800003, 16.710001, -13.530001, + -3.1800003, 16.710001, -13.530001, -3.1799994, 16.71, -13.529999, -3.1799994, 16.709997 + ], + [ + 23.63, 23.63, 23.630003, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, + 23.630001, 23.630001, 23.630001, 23.63, 23.630001, 23.630001, 23.63, 23.630001, + 23.630001, 23.63, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, 23.630003 + ], + [ + 0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0, + 0 + ]] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp new file mode 100644 index 000000000000..fe60c71eb8fa --- /dev/null +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -0,0 +1,184 @@ +#include "config.h" + +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wold-style-cast" +# pragma clang diagnostic ignored "-Wshadow" +# pragma clang diagnostic ignored "-Wimplicit-float-conversion" +#endif + +#include + +#ifdef __clang__ +# pragma clang diagnostic pop +#endif + + +#if USE_SEASONAL +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int ILLEGAL_COLUMN; +} + +/* +Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) +*/ +class FunctionSeriesDecomposeSTL : public IFunction +{ +public: + static constexpr auto name = "seriesDecomposeSTL"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + std::string getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + bool isVariadic() const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"time-series", &isArray, nullptr, "Array"}, + {"period", &isNumber, nullptr, "Number"}, + }; + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(std::make_shared(std::make_shared())); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + ColumnPtr array_ptr = arguments[0].column; + const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + + const IColumn & src_data = array->getData(); + + Float64 period; + + + auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); + if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get()) + || checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + period = period_ptr->getUInt(0); + else if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + { + if ((period = period_ptr->getFloat64(0)) < 0) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal value {} for second argument of function {}. Should be a positive number", + period, + getName()); + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second argument of function {}", + arguments[1].column->getName(), + getName()); + + + std::vector seasonal; + std::vector trend; + std::vector residue; + + + if (executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue) + || executeNumber(src_data, period, seasonal, trend, residue)) + { + auto ret = ColumnFloat32::create(); + auto & res_data = ret->getData(); + + ColumnArray::ColumnOffsets::MutablePtr col_offsets = ColumnArray::ColumnOffsets::create(); + auto & col_offsets_data = col_offsets->getData(); + + auto root_offsets = ColumnArray::ColumnOffsets::create(); + auto & root_offsets_data = root_offsets->getData(); + + res_data.insert(res_data.end(), seasonal.begin(), seasonal.end()); + col_offsets_data.push_back(res_data.size()); + + res_data.insert(res_data.end(), trend.begin(), trend.end()); + col_offsets_data.push_back(res_data.size()); + + res_data.insert(res_data.end(), residue.begin(), residue.end()); + col_offsets_data.push_back(res_data.size()); + + root_offsets_data.push_back(col_offsets->size()); + + ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(ret), std::move(col_offsets)); + return ColumnArray::create(std::move(nested_array_col), std::move(root_offsets)); + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), + getName()); + } + + template + bool executeNumber( + const IColumn & src_data, + Float64 period, + std::vector & seasonal, + std::vector & trend, + std::vector & residue) const + { + const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data); + if (!src_data_concrete) + return false; + + const PaddedPODArray & src_vec = src_data_concrete->getData(); + + size_t len = src_vec.size(); + if (len < 4) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName()); + + std::vector src(src_vec.begin(), src_vec.end()); + + auto res = stl::params().fit(src, static_cast(period)); + + if (res.seasonal.empty()) + return false; + + seasonal = res.seasonal; + trend = res.trend; + residue = res.remainder; + return true; + } +}; +REGISTER_FUNCTION(seriesDecomposeSTL) +{ + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess)", + .categories{"Time series analysis"}}); +} +} +#endif diff --git a/tests/queries/0_stateless/02813_series_decompose.reference b/tests/queries/0_stateless/02813_series_decompose.reference new file mode 100644 index 000000000000..110635919fe5 --- /dev/null +++ b/tests/queries/0_stateless/02813_series_decompose.reference @@ -0,0 +1,2 @@ +[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]] +[[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7]] diff --git a/tests/queries/0_stateless/02813_series_decompose.sql b/tests/queries/0_stateless/02813_series_decompose.sql new file mode 100644 index 000000000000..9c3855310390 --- /dev/null +++ b/tests/queries/0_stateless/02813_series_decompose.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest + +SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3); +SELECT seriesDecomposeSTL([2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], 0); +SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5); -- { serverError ILLEGAL_COLUMN} +SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5.2); --{ serverError ILLEGAL_COLUMN} +SELECT seriesDecomposeSTL(); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT seriesDecomposeSTL([]); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT seriesDecomposeSTL([1,2,3], 2); --{ serverError BAD_ARGUMENTS} +SELECT seriesDecomposeSTL([2,2,2,3,3,3]); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From 6d58c9940812999f5eee5c80d4e8fee883d4afe6 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Wed, 22 Nov 2023 07:11:18 -0800 Subject: [PATCH 023/114] fixed the failing fasttest and style-check --- src/Functions/seriesDecomposeSTL.cpp | 45 +++++++++---------- .../aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index fe60c71eb8fa..0802ebc724e4 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -1,29 +1,28 @@ #include "config.h" -#ifdef __clang__ -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wold-style-cast" -# pragma clang diagnostic ignored "-Wshadow" -# pragma clang diagnostic ignored "-Wimplicit-float-conversion" -#endif - -#include - -#ifdef __clang__ -# pragma clang diagnostic pop -#endif - - #if USE_SEASONAL -#include -#include -#include -#include -#include -#include -#include -#include -#include +# ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wold-style-cast" +# pragma clang diagnostic ignored "-Wshadow" +# pragma clang diagnostic ignored "-Wimplicit-float-conversion" +# endif + +# include + +# ifdef __clang__ +# pragma clang diagnostic pop +# endif + +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB { diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c39143b40e0d..daa805e5715f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1790,6 +1790,7 @@ llvm loadDefaultCAFile localhost localread +loess logTrace logagent loghouse From c0cff7b4f3aa58e7ca4a3481643e696d1fb32e77 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Mon, 27 Nov 2023 09:51:05 -0800 Subject: [PATCH 024/114] handled edge cases and addresses review comments --- contrib/stl-cpp-cmake/CMakeLists.txt | 7 +- .../functions/time-series-functions.md | 3 +- src/CMakeLists.txt | 4 +- src/Functions/CMakeLists.txt | 4 +- src/Functions/seriesDecomposeSTL.cpp | 72 ++++++++++--------- src/configure_config.cmake | 2 +- .../0_stateless/02813_series_decompose.sql | 5 +- 7 files changed, 54 insertions(+), 43 deletions(-) diff --git a/contrib/stl-cpp-cmake/CMakeLists.txt b/contrib/stl-cpp-cmake/CMakeLists.txt index cd04783beb19..6108a1c7b378 100644 --- a/contrib/stl-cpp-cmake/CMakeLists.txt +++ b/contrib/stl-cpp-cmake/CMakeLists.txt @@ -1,10 +1,9 @@ -option (ENABLE_SEASONAL "Enable stl-cpp" ${ENABLE_LIBRARIES}) +option (ENABLE_SEASONAL "Enable seasonal decompose (stl-cpp) library" ${ENABLE_LIBRARIES}) if (NOT ENABLE_SEASONAL) - message(STATUS "Not using stl-cpp") + message(STATUS "Not using seasonal decompose (stl-cpp)") return() endif() -#set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/stl-cpp") add_library(_stl-cpp INTERFACE) target_include_directories(_stl-cpp INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/stl-cpp/include) -add_library(ch_contrib::stl-cpp ALIAS _stl-cpp) +add_library(ch_contrib::seasonal-decompose ALIAS _stl-cpp) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 627744836a0d..3f5dea6b4c5e 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -61,8 +61,9 @@ Result: ## seriesDecomposeSTL Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) -Returns an array of arrays where the first array include seasonal components, the second array - trend, +Returns an array of three arrays where the first array include seasonal components, the second array - trend, and the third array - residue component. +https://www.wessa.net/download/stl.pdf **Syntax** diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b04b1575b18e..df490dc12b1a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -396,8 +396,8 @@ if (TARGET ch_contrib::nuraft) target_link_libraries (clickhouse_compression PUBLIC ch_contrib::nuraft) endif() -if (TARGET ch_contrib::stl-cpp) - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::stl-cpp) +if (TARGET ch_contrib::seasonal-decompose) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::seasonal-decompose) endif () dbms_target_link_libraries ( diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 8133356f8288..4f9b9176b890 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -75,8 +75,8 @@ if (TARGET ch_contrib::base64) list (APPEND PRIVATE_LIBS ch_contrib::base64) endif() -if (TARGET ch_contrib::stl-cpp) - list (APPEND PRIVATE_LIBS ch_contrib::stl-cpp) +if (TARGET ch_contrib::seasonal-decompose) + list (APPEND PRIVATE_LIBS ch_contrib::seasonal-decompose) endif() if (ENABLE_NLP) diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 0802ebc724e4..0c05d22a5ae0 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -1,28 +1,28 @@ #include "config.h" #if USE_SEASONAL -# ifdef __clang__ -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wold-style-cast" -# pragma clang diagnostic ignored "-Wshadow" -# pragma clang diagnostic ignored "-Wimplicit-float-conversion" -# endif - -# include - -# ifdef __clang__ -# pragma clang diagnostic pop -# endif - -# include -# include -# include -# include -# include -# include -# include -# include -# include +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wold-style-cast" +#pragma clang diagnostic ignored "-Wshadow" +#pragma clang diagnostic ignored "-Wimplicit-float-conversion" +#endif + +#include + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -46,8 +46,6 @@ class FunctionSeriesDecomposeSTL : public IFunction size_t getNumberOfArguments() const override { return 2; } - bool isVariadic() const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } @@ -56,7 +54,7 @@ class FunctionSeriesDecomposeSTL : public IFunction { FunctionArgumentDescriptors args{ {"time-series", &isArray, nullptr, "Array"}, - {"period", &isNumber, nullptr, "Number"}, + {"period", &isNativeNumber, nullptr, "Number"}, }; validateFunctionArgumentTypes(*this, arguments, args); @@ -158,18 +156,28 @@ class FunctionSeriesDecomposeSTL : public IFunction size_t len = src_vec.size(); if (len < 4) throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName()); + else if (period > (len / 2)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The series should have data of at least two period lengths for function {}", getName()); std::vector src(src_vec.begin(), src_vec.end()); - auto res = stl::params().fit(src, static_cast(period)); + try + { + auto res = stl::params().fit(src, static_cast(period)); - if (res.seasonal.empty()) - return false; + if (res.seasonal.empty()) + return false; - seasonal = res.seasonal; - trend = res.trend; - residue = res.remainder; - return true; + seasonal = res.seasonal; + trend = res.trend; + residue = res.remainder; + return true; + } + catch (const std::exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, e.what()); + } } }; REGISTER_FUNCTION(seriesDecomposeSTL) diff --git a/src/configure_config.cmake b/src/configure_config.cmake index c003ec466b77..030c4176153d 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -161,7 +161,7 @@ endif () if (ENABLE_OPENSSL) set(USE_OPENSSL_INTREE 1) endif () -if (TARGET ch_contrib::stl-cpp) +if (TARGET ch_contrib::seasonal-decompose) set(USE_SEASONAL 1) endif() if (TARGET ch_contrib::fiu) diff --git a/tests/queries/0_stateless/02813_series_decompose.sql b/tests/queries/0_stateless/02813_series_decompose.sql index 9c3855310390..eae9a84117b9 100644 --- a/tests/queries/0_stateless/02813_series_decompose.sql +++ b/tests/queries/0_stateless/02813_series_decompose.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-cpu-aarch64 +-- Tag no-cpu-aarch64: values generated are slighly different on aarch64 SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3); SELECT seriesDecomposeSTL([2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], 0); @@ -8,3 +9,5 @@ SELECT seriesDecomposeSTL(); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT seriesDecomposeSTL([]); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT seriesDecomposeSTL([1,2,3], 2); --{ serverError BAD_ARGUMENTS} SELECT seriesDecomposeSTL([2,2,2,3,3,3]); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT seriesDecomposeSTL([2,2,2,3,3,3], 9272653446478); --{ serverError BAD_ARGUMENTS} +SELECT seriesDecomposeSTL([2,2,2,3,3,3], 7); --{ serverError BAD_ARGUMENTS} From d4eb8c7352bc3ae732e1177f6d5afc8350408303 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Tue, 28 Nov 2023 10:06:40 -0800 Subject: [PATCH 025/114] handled NaN and inf values --- src/Functions/seriesDecomposeSTL.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 0c05d22a5ae0..7a9937e2b355 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -23,6 +23,8 @@ #include #include #include +#include + namespace DB { @@ -77,7 +79,8 @@ class FunctionSeriesDecomposeSTL : public IFunction period = period_ptr->getUInt(0); else if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) { - if ((period = period_ptr->getFloat64(0)) < 0) + period = period_ptr->getFloat64(0); + if (isNaN(period) || !std::isfinite(period) || period < 0) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Illegal value {} for second argument of function {}. Should be a positive number", @@ -164,7 +167,7 @@ class FunctionSeriesDecomposeSTL : public IFunction try { - auto res = stl::params().fit(src, static_cast(period)); + auto res = stl::params().fit(src, static_cast(std::round(period))); if (res.seasonal.empty()) return false; From c4ebc752099156b7945ac490125f1c13d52a54b3 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Thu, 14 Dec 2023 07:43:20 -0800 Subject: [PATCH 026/114] remove stl-cpp submodule --- contrib/CMakeLists.txt | 1 - contrib/stl-cpp | 1 - contrib/stl-cpp-cmake/CMakeLists.txt | 9 - src/CMakeLists.txt | 4 - src/Functions/CMakeLists.txt | 4 - src/Functions/seriesDecomposeSTL.cpp | 128 +++---- src/Functions/stl.hpp | 510 +++++++++++++++++++++++++++ src/configure_config.cmake | 3 - 8 files changed, 578 insertions(+), 82 deletions(-) delete mode 160000 contrib/stl-cpp delete mode 100644 contrib/stl-cpp-cmake/CMakeLists.txt create mode 100644 src/Functions/stl.hpp diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 84e3e8c0c7e4..1b5ba15187f8 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -77,7 +77,6 @@ add_contrib (minizip-ng-cmake minizip-ng) add_contrib (snappy-cmake snappy) add_contrib (rocksdb-cmake rocksdb) add_contrib (thrift-cmake thrift) -add_contrib (stl-cpp-cmake stl-cpp) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion add_contrib (avro-cmake avro) # requires: snappy diff --git a/contrib/stl-cpp b/contrib/stl-cpp deleted file mode 160000 index f3e1cd02b4f0..000000000000 --- a/contrib/stl-cpp +++ /dev/null @@ -1 +0,0 @@ -Subproject commit f3e1cd02b4f0341a7539ed1a171b21b14b37ed51 diff --git a/contrib/stl-cpp-cmake/CMakeLists.txt b/contrib/stl-cpp-cmake/CMakeLists.txt deleted file mode 100644 index 6108a1c7b378..000000000000 --- a/contrib/stl-cpp-cmake/CMakeLists.txt +++ /dev/null @@ -1,9 +0,0 @@ -option (ENABLE_SEASONAL "Enable seasonal decompose (stl-cpp) library" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_SEASONAL) - message(STATUS "Not using seasonal decompose (stl-cpp)") - return() -endif() -add_library(_stl-cpp INTERFACE) -target_include_directories(_stl-cpp INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/stl-cpp/include) -add_library(ch_contrib::seasonal-decompose ALIAS _stl-cpp) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index df490dc12b1a..86cb9acd0565 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -396,10 +396,6 @@ if (TARGET ch_contrib::nuraft) target_link_libraries (clickhouse_compression PUBLIC ch_contrib::nuraft) endif() -if (TARGET ch_contrib::seasonal-decompose) - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::seasonal-decompose) -endif () - dbms_target_link_libraries ( PRIVATE boost::filesystem diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 4f9b9176b890..896765945817 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -75,10 +75,6 @@ if (TARGET ch_contrib::base64) list (APPEND PRIVATE_LIBS ch_contrib::base64) endif() -if (TARGET ch_contrib::seasonal-decompose) - list (APPEND PRIVATE_LIBS ch_contrib::seasonal-decompose) -endif() - if (ENABLE_NLP) list (APPEND PRIVATE_LIBS ch_contrib::cld2) endif() diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 7a9937e2b355..eaa6619aae3e 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -1,6 +1,5 @@ #include "config.h" -#if USE_SEASONAL #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wold-style-cast" @@ -8,7 +7,7 @@ #pragma clang diagnostic ignored "-Wimplicit-float-conversion" #endif -#include +#include #ifdef __clang__ #pragma clang diagnostic pop @@ -69,83 +68,92 @@ class FunctionSeriesDecomposeSTL : public IFunction const ColumnArray * array = checkAndGetColumn(array_ptr.get()); const IColumn & src_data = array->getData(); + const ColumnArray::Offsets & src_offsets = array->getOffsets(); Float64 period; + auto ret = ColumnFloat32::create(); + auto & res_data = ret->getData(); - auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); - if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get()) - || checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) - period = period_ptr->getUInt(0); - else if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + ColumnArray::ColumnOffsets::MutablePtr col_offsets = ColumnArray::ColumnOffsets::create(); + auto & col_offsets_data = col_offsets->getData(); + + auto root_offsets = ColumnArray::ColumnOffsets::create(); + auto & root_offsets_data = root_offsets->getData(); + + ColumnArray::Offset prev_src_offset = 0; + + for (size_t i = 0; i < src_offsets.size(); ++i) { - period = period_ptr->getFloat64(0); - if (isNaN(period) || !std::isfinite(period) || period < 0) + auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); + if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get()) + || checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + period = period_ptr->getUInt(i); + else if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + { + period = period_ptr->getFloat64(i); + if (isNaN(period) || !std::isfinite(period) || period < 0) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal value {} for second argument of function {}. Should be a positive number", + period, + getName()); + } + else throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal value {} for second argument of function {}. Should be a positive number", - period, + "Illegal column {} of second argument of function {}", + arguments[1].column->getName(), getName()); - } - else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second argument of function {}", - arguments[1].column->getName(), - getName()); - - - std::vector seasonal; - std::vector trend; - std::vector residue; - - - if (executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue) - || executeNumber(src_data, period, seasonal, trend, residue)) - { - auto ret = ColumnFloat32::create(); - auto & res_data = ret->getData(); - ColumnArray::ColumnOffsets::MutablePtr col_offsets = ColumnArray::ColumnOffsets::create(); - auto & col_offsets_data = col_offsets->getData(); - auto root_offsets = ColumnArray::ColumnOffsets::create(); - auto & root_offsets_data = root_offsets->getData(); + std::vector seasonal; + std::vector trend; + std::vector residue; - res_data.insert(res_data.end(), seasonal.begin(), seasonal.end()); - col_offsets_data.push_back(res_data.size()); + ColumnArray::Offset curr_offset = src_offsets[i]; - res_data.insert(res_data.end(), trend.begin(), trend.end()); - col_offsets_data.push_back(res_data.size()); + if (executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) + || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)) + { + res_data.insert(res_data.end(), seasonal.begin(), seasonal.end()); + col_offsets_data.push_back(res_data.size()); - res_data.insert(res_data.end(), residue.begin(), residue.end()); - col_offsets_data.push_back(res_data.size()); + res_data.insert(res_data.end(), trend.begin(), trend.end()); + col_offsets_data.push_back(res_data.size()); - root_offsets_data.push_back(col_offsets->size()); + res_data.insert(res_data.end(), residue.begin(), residue.end()); + col_offsets_data.push_back(res_data.size()); - ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(ret), std::move(col_offsets)); - return ColumnArray::create(std::move(nested_array_col), std::move(root_offsets)); + root_offsets_data.push_back(col_offsets->size()); + + prev_src_offset = curr_offset; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), + getName()); } - else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), - getName()); + ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(ret), std::move(col_offsets)); + return ColumnArray::create(std::move(nested_array_col), std::move(root_offsets)); } template bool executeNumber( const IColumn & src_data, Float64 period, + ColumnArray::Offset & start, + ColumnArray::Offset & end, std::vector & seasonal, std::vector & trend, std::vector & residue) const @@ -156,14 +164,15 @@ class FunctionSeriesDecomposeSTL : public IFunction const PaddedPODArray & src_vec = src_data_concrete->getData(); - size_t len = src_vec.size(); + chassert(start <= end); + size_t len = end - start; if (len < 4) throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName()); else if (period > (len / 2)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The series should have data of at least two period lengths for function {}", getName()); - std::vector src(src_vec.begin(), src_vec.end()); + std::vector src((src_vec.begin() + start), (src_vec.begin() + end)); try { @@ -191,4 +200,3 @@ Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure B .categories{"Time series analysis"}}); } } -#endif diff --git a/src/Functions/stl.hpp b/src/Functions/stl.hpp new file mode 100644 index 000000000000..5b257b1c59de --- /dev/null +++ b/src/Functions/stl.hpp @@ -0,0 +1,510 @@ +/*! + * STL C++ v0.1.3 + * https://github.com/ankane/stl-cpp + * Unlicense OR MIT License + * + * Ported from https://www.netlib.org/a/stl + * + * Cleveland, R. B., Cleveland, W. S., McRae, J. E., & Terpenning, I. (1990). + * STL: A Seasonal-Trend Decomposition Procedure Based on Loess. + * Journal of Official Statistics, 6(1), 3-33. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace stl { + +bool est(const float* y, size_t n, size_t len, int ideg, float xs, float* ys, size_t nleft, size_t nright, float* w, bool userw, const float* rw) { + auto range = ((float) n) - 1.0; + auto h = std::max(xs - ((float) nleft), ((float) nright) - xs); + + if (len > n) { + h += (float) ((len - n) / 2); + } + + auto h9 = 0.999 * h; + auto h1 = 0.001 * h; + + // compute weights + auto a = 0.0; + for (auto j = nleft; j <= nright; j++) { + w[j - 1] = 0.0; + auto r = fabs(((float) j) - xs); + if (r <= h9) { + if (r <= h1) { + w[j - 1] = 1.0; + } else { + w[j - 1] = pow(1.0 - pow(r / h, 3), 3); + } + if (userw) { + w[j - 1] *= rw[j - 1]; + } + a += w[j - 1]; + } + } + + if (a <= 0.0) { + return false; + } else { // weighted least squares + for (auto j = nleft; j <= nright; j++) { // make sum of w(j) == 1 + w[j - 1] /= a; + } + + if (h > 0.0 && ideg > 0) { // use linear fit + auto a = 0.0; + for (auto j = nleft; j <= nright; j++) { // weighted center of x values + a += w[j - 1] * ((float) j); + } + auto b = xs - a; + auto c = 0.0; + for (auto j = nleft; j <= nright; j++) { + c += w[j - 1] * pow(((float) j) - a, 2); + } + if (sqrt(c) > 0.001 * range) { + b /= c; + + // points are spread out enough to compute slope + for (auto j = nleft; j <= nright; j++) { + w[j - 1] *= b * (((float) j) - a) + 1.0; + } + } + } + + *ys = 0.0; + for (auto j = nleft; j <= nright; j++) { + *ys += w[j - 1] * y[j - 1]; + } + + return true; + } +} + +void ess(const float* y, size_t n, size_t len, int ideg, size_t njump, bool userw, const float* rw, float* ys, float* res) { + if (n < 2) { + ys[0] = y[0]; + return; + } + + size_t nleft = 0; + size_t nright = 0; + + auto newnj = std::min(njump, n - 1); + if (len >= n) { + nleft = 1; + nright = n; + for (size_t i = 1; i <= n; i += newnj) { + auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw); + if (!ok) { + ys[i - 1] = y[i - 1]; + } + } + } else if (newnj == 1) { // newnj equal to one, len less than n + auto nsh = (len + 1) / 2; + nleft = 1; + nright = len; + for (size_t i = 1; i <= n; i++) { // fitted value at i + if (i > nsh && nright != n) { + nleft += 1; + nright += 1; + } + auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw); + if (!ok) { + ys[i - 1] = y[i - 1]; + } + } + } else { // newnj greater than one, len less than n + auto nsh = (len + 1) / 2; + for (size_t i = 1; i <= n; i += newnj) { // fitted value at i + if (i < nsh) { + nleft = 1; + nright = len; + } else if (i >= n - nsh + 1) { + nleft = n - len + 1; + nright = n; + } else { + nleft = i - nsh + 1; + nright = len + i - nsh; + } + auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw); + if (!ok) { + ys[i - 1] = y[i - 1]; + } + } + } + + if (newnj != 1) { + for (size_t i = 1; i <= n - newnj; i += newnj) { + auto delta = (ys[i + newnj - 1] - ys[i - 1]) / ((float) newnj); + for (auto j = i + 1; j <= i + newnj - 1; j++) { + ys[j - 1] = ys[i - 1] + delta * ((float) (j - i)); + } + } + auto k = ((n - 1) / newnj) * newnj + 1; + if (k != n) { + auto ok = est(y, n, len, ideg, (float) n, &ys[n - 1], nleft, nright, res, userw, rw); + if (!ok) { + ys[n - 1] = y[n - 1]; + } + if (k != n - 1) { + auto delta = (ys[n - 1] - ys[k - 1]) / ((float) (n - k)); + for (auto j = k + 1; j <= n - 1; j++) { + ys[j - 1] = ys[k - 1] + delta * ((float) (j - k)); + } + } + } + } +} + +void ma(const float* x, size_t n, size_t len, float* ave) { + auto newn = n - len + 1; + auto flen = (float) len; + auto v = 0.0; + + // get the first average + for (size_t i = 0; i < len; i++) { + v += x[i]; + } + + ave[0] = v / flen; + if (newn > 1) { + auto k = len; + auto m = 0; + for (size_t j = 1; j < newn; j++) { + // window down the array + v = v - x[m] + x[k]; + ave[j] = v / flen; + k += 1; + m += 1; + } + } +} + +void fts(const float* x, size_t n, size_t np, float* trend, float* work) { + ma(x, n, np, trend); + ma(trend, n - np + 1, np, work); + ma(work, n - 2 * np + 2, 3, trend); +} + +void rwts(const float* y, size_t n, const float* fit, float* rw) { + for (size_t i = 0; i < n; i++) { + rw[i] = fabs(y[i] - fit[i]); + } + + auto mid1 = (n - 1) / 2; + auto mid2 = n / 2; + + // sort + std::sort(rw, rw + n); + + auto cmad = 3.0 * (rw[mid1] + rw[mid2]); // 6 * median abs resid + auto c9 = 0.999 * cmad; + auto c1 = 0.001 * cmad; + + for (size_t i = 0; i < n; i++) { + auto r = fabs(y[i] - fit[i]); + if (r <= c1) { + rw[i] = 1.0; + } else if (r <= c9) { + rw[i] = pow(1.0 - pow(r / cmad, 2), 2); + } else { + rw[i] = 0.0; + } + } +} + +void ss(const float* y, size_t n, size_t np, size_t ns, int isdeg, size_t nsjump, bool userw, float* rw, float* season, float* work1, float* work2, float* work3, float* work4) { + for (size_t j = 1; j <= np; j++) { + size_t k = (n - j) / np + 1; + + for (size_t i = 1; i <= k; i++) { + work1[i - 1] = y[(i - 1) * np + j - 1]; + } + if (userw) { + for (size_t i = 1; i <= k; i++) { + work3[i - 1] = rw[(i - 1) * np + j - 1]; + } + } + ess(work1, k, ns, isdeg, nsjump, userw, work3, work2 + 1, work4); + auto xs = 0.0; + auto nright = std::min(ns, k); + auto ok = est(work1, k, ns, isdeg, xs, &work2[0], 1, nright, work4, userw, work3); + if (!ok) { + work2[0] = work2[1]; + } + xs = k + 1; + size_t nleft = std::max(1, (int) k - (int) ns + 1); + ok = est(work1, k, ns, isdeg, xs, &work2[k + 1], nleft, k, work4, userw, work3); + if (!ok) { + work2[k + 1] = work2[k]; + } + for (size_t m = 1; m <= k + 2; m++) { + season[(m - 1) * np + j - 1] = work2[m - 1]; + } + } +} + +void onestp(const float* y, size_t n, size_t np, size_t ns, size_t nt, size_t nl, int isdeg, int itdeg, int ildeg, size_t nsjump, size_t ntjump, size_t nljump, size_t ni, bool userw, float* rw, float* season, float* trend, float* work1, float* work2, float* work3, float* work4, float* work5) { + for (size_t j = 0; j < ni; j++) { + for (size_t i = 0; i < n; i++) { + work1[i] = y[i] - trend[i]; + } + + ss(work1, n, np, ns, isdeg, nsjump, userw, rw, work2, work3, work4, work5, season); + fts(work2, n + 2 * np, np, work3, work1); + ess(work3, n, nl, ildeg, nljump, false, work4, work1, work5); + for (size_t i = 0; i < n; i++) { + season[i] = work2[np + i] - work1[i]; + } + for (size_t i = 0; i < n; i++) { + work1[i] = y[i] - season[i]; + } + ess(work1, n, nt, itdeg, ntjump, userw, rw, trend, work3); + } +} + +void stl(const float* y, size_t n, size_t np, size_t ns, size_t nt, size_t nl, int isdeg, int itdeg, int ildeg, size_t nsjump, size_t ntjump, size_t nljump, size_t ni, size_t no, float* rw, float* season, float* trend) { + if (ns < 3) { + throw std::invalid_argument("seasonal_length must be at least 3"); + } + if (nt < 3) { + throw std::invalid_argument("trend_length must be at least 3"); + } + if (nl < 3) { + throw std::invalid_argument("low_pass_length must be at least 3"); + } + if (np < 2) { + throw std::invalid_argument("period must be at least 2"); + } + + if (isdeg != 0 && isdeg != 1) { + throw std::invalid_argument("seasonal_degree must be 0 or 1"); + } + if (itdeg != 0 && itdeg != 1) { + throw std::invalid_argument("trend_degree must be 0 or 1"); + } + if (ildeg != 0 && ildeg != 1) { + throw std::invalid_argument("low_pass_degree must be 0 or 1"); + } + + if (ns % 2 != 1) { + throw std::invalid_argument("seasonal_length must be odd"); + } + if (nt % 2 != 1) { + throw std::invalid_argument("trend_length must be odd"); + } + if (nl % 2 != 1) { + throw std::invalid_argument("low_pass_length must be odd"); + } + + auto work1 = std::vector(n + 2 * np); + auto work2 = std::vector(n + 2 * np); + auto work3 = std::vector(n + 2 * np); + auto work4 = std::vector(n + 2 * np); + auto work5 = std::vector(n + 2 * np); + + auto userw = false; + size_t k = 0; + + while (true) { + onestp(y, n, np, ns, nt, nl, isdeg, itdeg, ildeg, nsjump, ntjump, nljump, ni, userw, rw, season, trend, work1.data(), work2.data(), work3.data(), work4.data(), work5.data()); + k += 1; + if (k > no) { + break; + } + for (size_t i = 0; i < n; i++) { + work1[i] = trend[i] + season[i]; + } + rwts(y, n, work1.data(), rw); + userw = true; + } + + if (no <= 0) { + for (size_t i = 0; i < n; i++) { + rw[i] = 1.0; + } + } +} + +float var(const std::vector& series) { + auto mean = std::accumulate(series.begin(), series.end(), 0.0) / series.size(); + std::vector tmp; + tmp.reserve(series.size()); + for (auto v : series) { + tmp.push_back(pow(v - mean, 2)); + } + return std::accumulate(tmp.begin(), tmp.end(), 0.0) / (series.size() - 1); +} + +float strength(const std::vector& component, const std::vector& remainder) { + std::vector sr; + sr.reserve(remainder.size()); + for (size_t i = 0; i < remainder.size(); i++) { + sr.push_back(component[i] + remainder[i]); + } + return std::max(0.0, 1.0 - var(remainder) / var(sr)); +} + +class StlResult { +public: + std::vector seasonal; + std::vector trend; + std::vector remainder; + std::vector weights; + + inline float seasonal_strength() { + return strength(seasonal, remainder); + } + + inline float trend_strength() { + return strength(trend, remainder); + } +}; + +class StlParams { + std::optional ns_ = std::nullopt; + std::optional nt_ = std::nullopt; + std::optional nl_ = std::nullopt; + int isdeg_ = 0; + int itdeg_ = 1; + std::optional ildeg_ = std::nullopt; + std::optional nsjump_ = std::nullopt; + std::optional ntjump_ = std::nullopt; + std::optional nljump_ = std::nullopt; + std::optional ni_ = std::nullopt; + std::optional no_ = std::nullopt; + bool robust_ = false; + +public: + inline StlParams seasonal_length(size_t ns) { + this->ns_ = ns; + return *this; + } + + inline StlParams trend_length(size_t nt) { + this->nt_ = nt; + return *this; + } + + inline StlParams low_pass_length(size_t nl) { + this->nl_ = nl; + return *this; + } + + inline StlParams seasonal_degree(int isdeg) { + this->isdeg_ = isdeg; + return *this; + } + + inline StlParams trend_degree(int itdeg) { + this->itdeg_ = itdeg; + return *this; + } + + inline StlParams low_pass_degree(int ildeg) { + this->ildeg_ = ildeg; + return *this; + } + + inline StlParams seasonal_jump(size_t nsjump) { + this->nsjump_ = nsjump; + return *this; + } + + inline StlParams trend_jump(size_t ntjump) { + this->ntjump_ = ntjump; + return *this; + } + + inline StlParams low_pass_jump(size_t nljump) { + this->nljump_ = nljump; + return *this; + } + + inline StlParams inner_loops(bool ni) { + this->ni_ = ni; + return *this; + } + + inline StlParams outer_loops(bool no) { + this->no_ = no; + return *this; + } + + inline StlParams robust(bool robust) { + this->robust_ = robust; + return *this; + } + + StlResult fit(const float* y, size_t n, size_t np); + StlResult fit(const std::vector& y, size_t np); +}; + +StlParams params() { + return StlParams(); +} + +StlResult StlParams::fit(const float* y, size_t n, size_t np) { + if (n < 2 * np) { + throw std::invalid_argument("series has less than two periods"); + } + + auto ns = this->ns_.value_or(np); + + auto isdeg = this->isdeg_; + auto itdeg = this->itdeg_; + + auto res = StlResult { + std::vector(n), + std::vector(n), + std::vector(), + std::vector(n) + }; + + auto ildeg = this->ildeg_.value_or(itdeg); + auto newns = std::max(ns, (size_t) 3); + if (newns % 2 == 0) { + newns += 1; + } + + auto newnp = std::max(np, (size_t) 2); + auto nt = (size_t) ceil((1.5 * newnp) / (1.0 - 1.5 / (float) newns)); + nt = this->nt_.value_or(nt); + nt = std::max(nt, (size_t) 3); + if (nt % 2 == 0) { + nt += 1; + } + + auto nl = this->nl_.value_or(newnp); + if (nl % 2 == 0 && !this->nl_.has_value()) { + nl += 1; + } + + auto ni = this->ni_.value_or(this->robust_ ? 1 : 2); + auto no = this->no_.value_or(this->robust_ ? 15 : 0); + + auto nsjump = this->nsjump_.value_or((size_t) ceil(((float) newns) / 10.0)); + auto ntjump = this->ntjump_.value_or((size_t) ceil(((float) nt) / 10.0)); + auto nljump = this->nljump_.value_or((size_t) ceil(((float) nl) / 10.0)); + + stl(y, n, newnp, newns, nt, nl, isdeg, itdeg, ildeg, nsjump, ntjump, nljump, ni, no, res.weights.data(), res.seasonal.data(), res.trend.data()); + + res.remainder.reserve(n); + for (size_t i = 0; i < n; i++) { + res.remainder.push_back(y[i] - res.seasonal[i] - res.trend[i]); + } + + return res; +} + +StlResult StlParams::fit(const std::vector& y, size_t np) { + return StlParams::fit(y.data(), y.size(), np); +} + +} diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 030c4176153d..9358abdf7f8a 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -161,9 +161,6 @@ endif () if (ENABLE_OPENSSL) set(USE_OPENSSL_INTREE 1) endif () -if (TARGET ch_contrib::seasonal-decompose) - set(USE_SEASONAL 1) -endif() if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() From 3502245ecb6532f36355aeac3214ddb9c2973c3e Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 04:32:33 +0000 Subject: [PATCH 027/114] use fs::absolute instead of fs::canonical To allow users to work with symlinks in user_files_path Signed-off-by: Duc Canh Le --- src/Storages/StorageFile.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 25bb6691ff68..b496b5a5e2e1 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -115,7 +115,9 @@ void listFilesWithRegexpMatchingImpl( { try { - fs::path path = fs::canonical(path_for_ls + for_match); + /// Do not use fs::canonical or fs::weakly_canonical. + /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. + fs::path path = fs::absolute(path_for_ls + for_match); result.push_back(path.string()); } catch (const std::exception &) // NOLINT From 35325e8e830be18e3f010ea305099c7196b0fb93 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 04:57:41 +0000 Subject: [PATCH 028/114] add a test Signed-off-by: Duc Canh Le --- .../0_stateless/02051_symlinks_to_user_files.reference | 1 + tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference index d86bac9de59a..2c94e4837100 100644 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference @@ -1 +1,2 @@ OK +OK diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 0af71e4deee3..a88c0ddd5e90 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -8,13 +8,14 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # See 01658_read_file_to_string_column.sh user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_PATH="${user_files_path}/file/" +FILE_PATH="${user_files_path}/file" mkdir -p ${FILE_PATH} chmod 777 ${FILE_PATH} FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${FILE_PATH}/${FILE} +symlink_path_with_regex=${FILE_PATH}*/${FILE} file_path=$CUR_DIR/${FILE} touch ${file_path} @@ -29,3 +30,4 @@ trap cleanup EXIT ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; ${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String') order by a"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path_with_regex}', 'Values', 'a String') order by a"; From 2186aa8f217cc83374f5db6bdab089662ba8085a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 08:03:01 +0000 Subject: [PATCH 029/114] Revert "Revert "Implement punycode encoding/decoding"" This reverts commit 345d29a3c1782254ad05d7ab9352a570386579ef. --- .gitmodules | 3 + contrib/CMakeLists.txt | 1 + contrib/idna | 1 + contrib/idna-cmake/CMakeLists.txt | 24 +++ .../functions/string-functions.md | 65 +++++++ src/Common/config.h.in | 1 + src/Functions/CMakeLists.txt | 4 + src/Functions/FunctionSqid.cpp | 2 +- src/Functions/punycode.cpp | 165 ++++++++++++++++++ src/configure_config.cmake | 3 + .../0_stateless/02932_punycode.reference | 35 ++++ tests/queries/0_stateless/02932_punycode.sql | 63 +++++++ .../aspell-ignore/en/aspell-dict.txt | 3 + 13 files changed, 369 insertions(+), 1 deletion(-) create mode 160000 contrib/idna create mode 100644 contrib/idna-cmake/CMakeLists.txt create mode 100644 src/Functions/punycode.cpp create mode 100644 tests/queries/0_stateless/02932_punycode.reference create mode 100644 tests/queries/0_stateless/02932_punycode.sql diff --git a/.gitmodules b/.gitmodules index 53ef899dd996..3b9faea3cc19 100644 --- a/.gitmodules +++ b/.gitmodules @@ -360,3 +360,6 @@ [submodule "contrib/sqids-cpp"] path = contrib/sqids-cpp url = https://github.com/sqids/sqids-cpp.git +[submodule "contrib/idna"] + path = contrib/idna + url = https://github.com/ada-url/idna.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1b5ba15187f8..02cb19d4c077 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -154,6 +154,7 @@ add_contrib (libpqxx-cmake libpqxx) add_contrib (libpq-cmake libpq) add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) +add_contrib (idna-cmake idna) add_contrib (datasketches-cpp-cmake datasketches-cpp) add_contrib (incbin-cmake incbin) add_contrib (sqids-cpp-cmake sqids-cpp) diff --git a/contrib/idna b/contrib/idna new file mode 160000 index 000000000000..3c8be01d42b7 --- /dev/null +++ b/contrib/idna @@ -0,0 +1 @@ +Subproject commit 3c8be01d42b75649f1ac9b697d0ef757eebfe667 diff --git a/contrib/idna-cmake/CMakeLists.txt b/contrib/idna-cmake/CMakeLists.txt new file mode 100644 index 000000000000..1138b836192a --- /dev/null +++ b/contrib/idna-cmake/CMakeLists.txt @@ -0,0 +1,24 @@ +option(ENABLE_IDNA "Enable idna support" ${ENABLE_LIBRARIES}) +if ((NOT ENABLE_IDNA)) + message (STATUS "Not using idna") + return() +endif() +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/idna") + +set (SRCS + "${LIBRARY_DIR}/src/idna.cpp" + "${LIBRARY_DIR}/src/mapping.cpp" + "${LIBRARY_DIR}/src/mapping_tables.cpp" + "${LIBRARY_DIR}/src/normalization.cpp" + "${LIBRARY_DIR}/src/normalization_tables.cpp" + "${LIBRARY_DIR}/src/punycode.cpp" + "${LIBRARY_DIR}/src/to_ascii.cpp" + "${LIBRARY_DIR}/src/to_unicode.cpp" + "${LIBRARY_DIR}/src/unicode_transcoding.cpp" + "${LIBRARY_DIR}/src/validity.cpp" +) + +add_library (_idna ${SRCS}) +target_include_directories(_idna PUBLIC "${LIBRARY_DIR}/include") + +add_library (ch_contrib::idna ALIAS _idna) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 206942119120..5bad7d53e62f 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1383,6 +1383,71 @@ Result: └──────────────────┘ ``` +## punycodeEncode + +Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) of a string. +The string must be UTF8-encoded, otherwise results are undefined. + +**Syntax** + +``` sql +punycodeEncode(val) +``` + +**Arguments** + +- `val` - Input value. [String](../data-types/string.md) + +**Returned value** + +- A Punycode representation of the input value. [String](../data-types/string.md) + +**Example** + +``` sql +select punycodeEncode('München'); +``` + +Result: + +```result +┌─punycodeEncode('München')─┐ +│ Mnchen-3ya │ +└───────────────────────────┘ +``` + +## punycodeDecode + +Returns the UTF8-encoded plaintext of a [Punycode](https://en.wikipedia.org/wiki/Punycode)-encoded string. + +**Syntax** + +``` sql +punycodeEncode(val) +``` + +**Arguments** + +- `val` - Punycode-encoded string. [String](../data-types/string.md) + +**Returned value** + +- The plaintext of the input value. [String](../data-types/string.md) + +**Example** + +``` sql +select punycodeDecode('Mnchen-3ya'); +``` + +Result: + +```result +┌─punycodeEncode('Mnchen-3ya')─┐ +│ München │ +└──────────────────────────────┘ +``` + ## byteHammingDistance Calculates the [hamming distance](https://en.wikipedia.org/wiki/Hamming_distance) between two byte strings. diff --git a/src/Common/config.h.in b/src/Common/config.h.in index f84e28942c54..5b3388a3b7d3 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -28,6 +28,7 @@ #cmakedefine01 USE_S2_GEOMETRY #cmakedefine01 USE_FASTOPS #cmakedefine01 USE_SQIDS +#cmakedefine01 USE_IDNA #cmakedefine01 USE_NLP #cmakedefine01 USE_VECTORSCAN #cmakedefine01 USE_LIBURING diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 896765945817..a06e898b7c5c 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -83,6 +83,10 @@ if (TARGET ch_contrib::sqids) list (APPEND PRIVATE_LIBS ch_contrib::sqids) endif() +if (TARGET ch_contrib::idna) + list (APPEND PRIVATE_LIBS ch_contrib::idna) +endif() + if (TARGET ch_contrib::h3) list (APPEND PRIVATE_LIBS ch_contrib::h3) endif() diff --git a/src/Functions/FunctionSqid.cpp b/src/Functions/FunctionSqid.cpp index 546263914c29..4517bba963eb 100644 --- a/src/Functions/FunctionSqid.cpp +++ b/src/Functions/FunctionSqid.cpp @@ -1,6 +1,6 @@ #include "config.h" -#ifdef ENABLE_SQIDS +#if USE_SQIDS #include #include diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp new file mode 100644 index 000000000000..c11409f0d1a7 --- /dev/null +++ b/src/Functions/punycode.cpp @@ -0,0 +1,165 @@ +#include "config.h" + +#if USE_IDNA + +#include +#include + +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wnewline-eof" +#endif +# include +# include +#ifdef __clang__ +# pragma clang diagnostic pop +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + +struct PunycodeEncodeImpl +{ + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::u32string value_utf32; + std::string value_puny; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + + const size_t value_utf32_length = ada::idna::utf32_length_from_utf8(value, value_length); + value_utf32.resize(value_utf32_length); + ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); + + const bool ok = ada::idna::utf32_to_punycode(value_utf32, value_puny); + if (!ok) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode encoding"); + + res_data.insert(value_puny.c_str(), value_puny.c_str() + value_puny.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + value_utf32.clear(); + value_puny.clear(); /// utf32_to_punycode() appends to its output string + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeEncode function"); + } +}; + +struct PunycodeDecodeImpl +{ + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::u32string value_utf32; + std::string value_utf8; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + + const std::string_view value_punycode(value, value_length); + const bool ok = ada::idna::punycode_to_utf32(value_punycode, value_utf32); + if (!ok) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode decoding"); + + const size_t utf8_length = ada::idna::utf8_length_from_utf32(value_utf32.data(), value_utf32.size()); + value_utf8.resize(utf8_length); + ada::idna::utf32_to_utf8(value_utf32.data(), value_utf32.size(), value_utf8.data()); + + res_data.insert(value_utf8.c_str(), value_utf8.c_str() + value_utf8.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + value_utf32.clear(); /// punycode_to_utf32() appends to its output string + value_utf8.clear(); + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeDecode function"); + } +}; + +struct NamePunycodeEncode +{ + static constexpr auto name = "punycodeEncode"; +}; + +struct NamePunycodeDecode +{ + static constexpr auto name = "punycodeDecode"; +}; + +REGISTER_FUNCTION(Punycode) +{ + factory.registerFunction>(FunctionDocumentation{ + .description=R"( +Computes a Punycode representation of a string.)", + .syntax="punycodeEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="The punycode representation [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT punycodeEncode('München') AS puny;", + R"( +┌─puny───────┐ +│ Mnchen-3ya │ +└────────────┘ + )" + }} + }); + + factory.registerFunction>(FunctionDocumentation{ + .description=R"( +Computes a Punycode representation of a string.)", + .syntax="punycodeDecode(str)", + .arguments={{"str", "A Punycode-encoded string"}}, + .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT punycodeDecode('Mnchen-3ya') AS plain;", + R"( +┌─plain───┐ +│ München │ +└─────────┘ + )" + }} + }); +} + +} + +#endif diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 9358abdf7f8a..7de2d5a9fdd1 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -129,6 +129,9 @@ endif() if (TARGET ch_contrib::sqids) set(USE_SQIDS 1) endif() +if (TARGET ch_contrib::idna) + set(USE_IDNA 1) +endif() if (TARGET ch_contrib::vectorscan) set(USE_VECTORSCAN 1) endif() diff --git a/tests/queries/0_stateless/02932_punycode.reference b/tests/queries/0_stateless/02932_punycode.reference new file mode 100644 index 000000000000..7a39a221e085 --- /dev/null +++ b/tests/queries/0_stateless/02932_punycode.reference @@ -0,0 +1,35 @@ +-- Negative tests +-- Regular cases +a a- a +A A- A +-- --- -- +London London- London +Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson +This has spaces This has spaces- This has spaces +-> $1.00 <- -> $1.00 <-- -> $1.00 <- +а 80a а +ü tda ü +α mxa α +例 fsq 例 +😉 n28h 😉 +αβγ mxacd αβγ +München Mnchen-3ya München +Mnchen-3ya Mnchen-3ya- Mnchen-3ya +München-Ost Mnchen-Ost-9db München-Ost +Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost +abæcdöef abcdef-qua4k abæcdöef +правда 80aafi6cg правда +ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข +ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 +MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 +「bücher」 bcher-kva8445foa 「bücher」 +团淄 3bs854c 团淄 +-- Special cases + + +\N +\N +Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +München Mnchen-3ya München +abc abc- abc +aäoöuü aou-qla5gqb aäoöuü diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql new file mode 100644 index 000000000000..dd18a43ecc9c --- /dev/null +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -0,0 +1,63 @@ +-- Tags: no-fasttest +-- no-fasttest: requires idna library + +SELECT '-- Negative tests'; + +SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } +SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } + +SELECT '-- Regular cases'; + +-- The test cases originate from the ada idna unit tests: +--- https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt + +SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; + +SELECT '-- Special cases'; + +SELECT punycodeDecode(''); +SELECT punycodeEncode(''); +SELECT punycodeDecode(NULL); +SELECT punycodeEncode(NULL); + +-- garbage Punycode-encoded values +SELECT punycodeDecode('no punycode'); -- { serverError BAD_ARGUMENTS } + +-- long input +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; + +-- non-const values +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (str String) ENGINE=MergeTree ORDER BY str; +INSERT INTO tab VALUES ('abc') ('aäoöuü') ('München'); +SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original FROM tab; +DROP TABLE tab; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d68330771e5b..f339f4518784 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -714,6 +714,7 @@ Promtail Protobuf ProtobufSingle ProxySQL +Punycode PyArrow PyCharm QEMU @@ -2075,6 +2076,8 @@ pseudorandom pseudorandomize psql ptrs +punycodeDecode +punycodeEncode pushdown pwrite py From 45613f1d14841756a15219c7ecb81aadd59496b9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 09:18:25 +0000 Subject: [PATCH 030/114] Refactor punycodeEncode/Decode() functions to inerit from IFunction directly --- src/Functions/punycode.cpp | 121 +++++++++++++++---- tests/queries/0_stateless/02932_punycode.sql | 4 +- 2 files changed, 100 insertions(+), 25 deletions(-) diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index c11409f0d1a7..fb89759b24d6 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -2,8 +2,11 @@ #if USE_IDNA +#include +#include #include -#include +#include +#include #ifdef __clang__ # pragma clang diagnostic push @@ -24,8 +27,57 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -struct PunycodeEncodeImpl +namespace { + +enum class ExceptionMode +{ + Throw, + Null +}; + +template +class FunctionPunycodeEncode : public IFunction +{ +public: + static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"str", &isString, nullptr, "String"}, + }; + validateFunctionArgumentTypes(*this, arguments, args); + + auto return_type = std::make_shared(); + + if constexpr (exception_mode == ExceptionMode::Null) + return makeNullable(return_type); + else + return return_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnPtr column = arguments[0].column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + return col_res; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments[0].column->getName(), getName()); + } + +private: static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -61,15 +113,50 @@ struct PunycodeEncodeImpl value_puny.clear(); /// utf32_to_punycode() appends to its output string } } +}; + +template +class FunctionPunycodeDecode : public IFunction +{ +public: + static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeDecodeOrNull" : "punycodeDecode"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"str", &isString, nullptr, "String"}, + }; + validateFunctionArgumentTypes(*this, arguments, args); + + auto return_type = std::make_shared(); + + if constexpr (exception_mode == ExceptionMode::Null) + return makeNullable(return_type); + else + return return_type; + } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeEncode function"); + const ColumnPtr column = arguments[0].column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + return col_res; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments[0].column->getName(), getName()); } -}; -struct PunycodeDecodeImpl -{ +private: static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -106,26 +193,13 @@ struct PunycodeDecodeImpl value_utf8.clear(); } } - - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeDecode function"); - } -}; - -struct NamePunycodeEncode -{ - static constexpr auto name = "punycodeEncode"; }; -struct NamePunycodeDecode -{ - static constexpr auto name = "punycodeDecode"; -}; +} REGISTER_FUNCTION(Punycode) { - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string.)", .syntax="punycodeEncode(str)", @@ -142,7 +216,7 @@ Computes a Punycode representation of a string.)", }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string.)", .syntax="punycodeDecode(str)", @@ -158,6 +232,7 @@ Computes a Punycode representation of a string.)", )" }} }); + } } diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql index dd18a43ecc9c..afc3c7e87129 100644 --- a/tests/queries/0_stateless/02932_punycode.sql +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -9,8 +9,8 @@ SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } -SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } +SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '-- Regular cases'; From 12fda5f3095edc9aae987391194c97482777f560 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 09:22:29 +0000 Subject: [PATCH 031/114] fix 02771_multidirectory_globs_storage_file Signed-off-by: Duc Canh Le --- src/Storages/StorageFile.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b496b5a5e2e1..b3c686c290dc 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -115,10 +115,12 @@ void listFilesWithRegexpMatchingImpl( { try { - /// Do not use fs::canonical or fs::weakly_canonical. + /// We use fs::canonical to check if the file exists but the result path + /// will be fs::absolute. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. - fs::path path = fs::absolute(path_for_ls + for_match); - result.push_back(path.string()); + fs::path canonical_path = fs::canonical(path_for_ls + for_match); + fs::path absolute_path = fs::absolute(path_for_ls + for_match); + result.push_back(absolute_path.string()); } catch (const std::exception &) // NOLINT { From b5d7ea2b0255e541e2b3dad539e90726cc894bac Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 10:50:00 +0000 Subject: [PATCH 032/114] Introduce OrNull() overloads for punycodeEncode/Decode() --- .../functions/string-functions.md | 11 +- src/Functions/punycode.cpp | 125 ++++++++++++++---- .../0_stateless/02932_punycode.reference | 67 ++++++---- tests/queries/0_stateless/02932_punycode.sql | 77 +++++++---- 4 files changed, 196 insertions(+), 84 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5bad7d53e62f..eb5cb29c5027 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1386,7 +1386,7 @@ Result: ## punycodeEncode Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) of a string. -The string must be UTF8-encoded, otherwise results are undefined. +The string must be UTF8-encoded, otherwise an exception is thrown. **Syntax** @@ -1416,9 +1416,14 @@ Result: └───────────────────────────┘ ``` +## punycodeEncodeOrNull + +Like `punycodeEncode` but returns `NULL` in case of an error instead of throwing an exception. + ## punycodeDecode Returns the UTF8-encoded plaintext of a [Punycode](https://en.wikipedia.org/wiki/Punycode)-encoded string. +If no valid Punycode-encoded string is given, an exception is thrown. **Syntax** @@ -1448,6 +1453,10 @@ Result: └──────────────────────────────┘ ``` +## punycodeDecodeOrNull + +Like `punycodeDecode` but returns `NULL` in case of an error instead of throwing an exception. + ## byteHammingDistance Calculates the [hamming distance](https://en.wikipedia.org/wiki/Hamming_distance) between two byte strings. diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index fb89759b24d6..ad66c723f053 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -2,7 +2,9 @@ #if USE_IDNA +#include #include +#include #include #include #include @@ -30,17 +32,17 @@ namespace ErrorCodes namespace { -enum class ExceptionMode +enum class ErrorHandling { Throw, Null }; -template +template class FunctionPunycodeEncode : public IFunction { public: - static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; + static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -57,7 +59,7 @@ class FunctionPunycodeEncode : public IFunction auto return_type = std::make_shared(); - if constexpr (exception_mode == ExceptionMode::Null) + if constexpr (error_handling == ErrorHandling::Null) return makeNullable(return_type); else return return_type; @@ -65,16 +67,18 @@ class FunctionPunycodeEncode : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnPtr column = arguments[0].column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - return col_res; - } + auto col_res = ColumnString::create(); + ColumnUInt8::MutablePtr col_res_null; + if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) + vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + + if constexpr (error_handling == ErrorHandling::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_res_null)); + else + return col_res; } private: @@ -82,11 +86,14 @@ class FunctionPunycodeEncode : public IFunction const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + ColumnUInt8::MutablePtr & col_res_null) { const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII res_offsets.reserve(rows); + if constexpr (error_handling == ErrorHandling::Null) + col_res_null = ColumnUInt8::create(rows, 0); size_t prev_offset = 0; std::u32string value_utf32; @@ -102,7 +109,17 @@ class FunctionPunycodeEncode : public IFunction const bool ok = ada::idna::utf32_to_punycode(value_utf32, value_puny); if (!ok) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode encoding"); + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); + } + else + { + value_puny.clear(); + col_res_null->getData()[row] = 1; + } + } res_data.insert(value_puny.c_str(), value_puny.c_str() + value_puny.size() + 1); res_offsets.push_back(res_data.size()); @@ -115,11 +132,11 @@ class FunctionPunycodeEncode : public IFunction } }; -template +template class FunctionPunycodeDecode : public IFunction { public: - static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeDecodeOrNull" : "punycodeDecode"; + static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeDecodeOrNull" : "punycodeDecode"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -136,7 +153,7 @@ class FunctionPunycodeDecode : public IFunction auto return_type = std::make_shared(); - if constexpr (exception_mode == ExceptionMode::Null) + if constexpr (error_handling == ErrorHandling::Null) return makeNullable(return_type); else return return_type; @@ -144,16 +161,19 @@ class FunctionPunycodeDecode : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnPtr column = arguments[0].column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - return col_res; - } + auto col_res = ColumnString::create(); + ColumnUInt8::MutablePtr col_res_null; + + if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) + vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + + if constexpr (error_handling == ErrorHandling::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_res_null)); + else + return col_res; } private: @@ -161,11 +181,14 @@ class FunctionPunycodeDecode : public IFunction const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + ColumnUInt8::MutablePtr & col_res_null) { const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII res_offsets.reserve(rows); + if constexpr (error_handling == ErrorHandling::Null) + col_res_null = ColumnUInt8::create(rows, 0); size_t prev_offset = 0; std::u32string value_utf32; @@ -178,7 +201,17 @@ class FunctionPunycodeDecode : public IFunction const std::string_view value_punycode(value, value_length); const bool ok = ada::idna::punycode_to_utf32(value_punycode, value_utf32); if (!ok) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode decoding"); + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' is not a valid Punycode-encoded string", value_punycode); + } + else + { + value_utf32.clear(); + col_res_null->getData()[row] = 1; + } + } const size_t utf8_length = ada::idna::utf8_length_from_utf32(value_utf32.data(), value_utf32.size()); value_utf8.resize(utf8_length); @@ -199,9 +232,9 @@ class FunctionPunycodeDecode : public IFunction REGISTER_FUNCTION(Punycode) { - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string.)", +Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeEncode(str)", .arguments={{"str", "Input string"}}, .returned_value="The punycode representation [String](/docs/en/sql-reference/data-types/string.md).", @@ -216,9 +249,26 @@ Computes a Punycode representation of a string.)", }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string.)", +Computes a Punycode representation of a string. Returns NULL in case of error)", + .syntax="punycodeEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="The punycode representation [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT punycodeEncodeOrNull('München') AS puny;", + R"( +┌─puny───────┐ +│ Mnchen-3ya │ +└────────────┘ + )" + }} + }); + + factory.registerFunction>(FunctionDocumentation{ + .description=R"( +Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeDecode(str)", .arguments={{"str", "A Punycode-encoded string"}}, .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", @@ -228,6 +278,23 @@ Computes a Punycode representation of a string.)", R"( ┌─plain───┐ │ München │ +└─────────┘ + )" + }} + }); + + factory.registerFunction>(FunctionDocumentation{ + .description=R"( +Computes a Punycode representation of a string. Returns NULL in case of error)", + .syntax="punycodeDecode(str)", + .arguments={{"str", "A Punycode-encoded string"}}, + .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT punycodeDecodeOrNull('Mnchen-3ya') AS plain;", + R"( +┌─plain───┐ +│ München │ └─────────┘ )" }} diff --git a/tests/queries/0_stateless/02932_punycode.reference b/tests/queries/0_stateless/02932_punycode.reference index 7a39a221e085..a722894bce8e 100644 --- a/tests/queries/0_stateless/02932_punycode.reference +++ b/tests/queries/0_stateless/02932_punycode.reference @@ -1,35 +1,46 @@ -- Negative tests -- Regular cases -a a- a -A A- A --- --- -- -London London- London -Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson -This has spaces This has spaces- This has spaces --> $1.00 <- -> $1.00 <-- -> $1.00 <- -а 80a а -ü tda ü -α mxa α -例 fsq 例 -😉 n28h 😉 -αβγ mxacd αβγ -München Mnchen-3ya München -Mnchen-3ya Mnchen-3ya- Mnchen-3ya -München-Ost Mnchen-Ost-9db München-Ost -Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost -abæcdöef abcdef-qua4k abæcdöef -правда 80aafi6cg правда -ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข -ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 -MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 -「bücher」 bcher-kva8445foa 「bücher」 -团淄 3bs854c 团淄 +a a- a a- a +A A- A A- A +-- --- -- --- -- +London London- London London- London +Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson +This has spaces This has spaces- This has spaces This has spaces- This has spaces +-> $1.00 <- -> $1.00 <-- -> $1.00 <- -> $1.00 <-- -> $1.00 <- +а 80a а 80a а +ü tda ü tda ü +α mxa α mxa α +例 fsq 例 fsq 例 +😉 n28h 😉 n28h 😉 +αβγ mxacd αβγ mxacd αβγ +München Mnchen-3ya München Mnchen-3ya München +Mnchen-3ya Mnchen-3ya- Mnchen-3ya Mnchen-3ya- Mnchen-3ya +München-Ost Mnchen-Ost-9db München-Ost Mnchen-Ost-9db München-Ost +Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost +abæcdöef abcdef-qua4k abæcdöef abcdef-qua4k abæcdöef +правда 80aafi6cg правда 80aafi6cg правда +ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข +ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 +MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 +「bücher」 bcher-kva8445foa 「bücher」 bcher-kva8445foa 「bücher」 +团淄 3bs854c 团淄 3bs854c 团淄 -- Special cases + + +\N +\N +\N \N \N -Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. -München Mnchen-3ya München -abc abc- abc -aäoöuü aou-qla5gqb aäoöuü +Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +München Mnchen-3ya München Mnchen-3ya München +abc abc- abc abc- abc +aäoöuü aou-qla5gqb aäoöuü aou-qla5gqb aäoöuü +Also no punycode \N +London- London +Mnchen-3ya München +No punycode \N +Rtting-3ya Rütting +XYZ no punycode \N diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql index afc3c7e87129..a142848d427b 100644 --- a/tests/queries/0_stateless/02932_punycode.sql +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -4,60 +4,85 @@ SELECT '-- Negative tests'; SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeEncodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeEncodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeEncodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeEncodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '-- Regular cases'; -- The test cases originate from the ada idna unit tests: --- https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt -SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; SELECT '-- Special cases'; SELECT punycodeDecode(''); +SELECT punycodeDecodeOrNull(''); SELECT punycodeEncode(''); +SELECT punycodeEncodeOrNull(''); + SELECT punycodeDecode(NULL); +SELECT punycodeDecodeOrNull(NULL); SELECT punycodeEncode(NULL); +SELECT punycodeEncodeOrNull(NULL); -- garbage Punycode-encoded values SELECT punycodeDecode('no punycode'); -- { serverError BAD_ARGUMENTS } +SELECT punycodeDecodeOrNull('no punycode'); -- long input -SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -- non-const values DROP TABLE IF EXISTS tab; CREATE TABLE tab (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO tab VALUES ('abc') ('aäoöuü') ('München'); -SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original FROM tab; +SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull FROM tab; +DROP TABLE tab; + +-- non-const values with a few invalid values for testing the OrNull variants +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (puny String) ENGINE=MergeTree ORDER BY puny; +INSERT INTO tab VALUES ('Also no punycode') ('London-') ('Mnchen-3ya') ('No punycode') ('Rtting-3ya') ('XYZ no punycode'); +SELECT puny, punycodeDecode(puny) AS original FROM tab; -- { serverError BAD_ARGUMENTS } +SELECT puny, punycodeDecodeOrNull(puny) AS original FROM tab; DROP TABLE tab; From 9a49f0cbb3a278e61d8390c8968aef920b309975 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 11:26:01 +0000 Subject: [PATCH 033/114] Factorize common code --- src/Functions/punycode.cpp | 78 ++++++++++++-------------------------- 1 file changed, 24 insertions(+), 54 deletions(-) diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index ad66c723f053..869b79daa10a 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -38,13 +38,14 @@ enum class ErrorHandling Null }; -template -class FunctionPunycodeEncode : public IFunction + +template +class FunctionPunycode : public IFunction { public: - static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; + static constexpr auto name = Impl::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared>(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } @@ -59,7 +60,7 @@ class FunctionPunycodeEncode : public IFunction auto return_type = std::make_shared(); - if constexpr (error_handling == ErrorHandling::Null) + if constexpr (Impl::error_handling == ErrorHandling::Null) return makeNullable(return_type); else return return_type; @@ -70,18 +71,25 @@ class FunctionPunycodeEncode : public IFunction auto col_res = ColumnString::create(); ColumnUInt8::MutablePtr col_res_null; if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) - vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); + Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); - if constexpr (error_handling == ErrorHandling::Null) + if constexpr (Impl::error_handling == ErrorHandling::Null) return ColumnNullable::create(std::move(col_res), std::move(col_res_null)); else return col_res; } +}; + + +template +struct PunycodeEncodeImpl +{ + static constexpr auto error_handling = error_handling_; + static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; -private: static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -132,51 +140,13 @@ class FunctionPunycodeEncode : public IFunction } }; -template -class FunctionPunycodeDecode : public IFunction + +template +struct PunycodeDecodeImpl { -public: + static constexpr auto error_handling = error_handling_; static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeDecodeOrNull" : "punycodeDecode"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"str", &isString, nullptr, "String"}, - }; - validateFunctionArgumentTypes(*this, arguments, args); - - auto return_type = std::make_shared(); - - if constexpr (error_handling == ErrorHandling::Null) - return makeNullable(return_type); - else - return return_type; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - auto col_res = ColumnString::create(); - ColumnUInt8::MutablePtr col_res_null; - - if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) - vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments[0].column->getName(), getName()); - - if constexpr (error_handling == ErrorHandling::Null) - return ColumnNullable::create(std::move(col_res), std::move(col_res_null)); - else - return col_res; - } - -private: static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -232,7 +202,7 @@ class FunctionPunycodeDecode : public IFunction REGISTER_FUNCTION(Punycode) { - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeEncode(str)", @@ -249,7 +219,7 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Returns NULL in case of error)", .syntax="punycodeEncode(str)", @@ -266,7 +236,7 @@ Computes a Punycode representation of a string. Returns NULL in case of error)", }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeDecode(str)", @@ -283,7 +253,7 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Returns NULL in case of error)", .syntax="punycodeDecode(str)", From 7dd128f90f2810ce640f841a05bd0733594ae1ec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jan 2024 16:00:50 +0000 Subject: [PATCH 034/114] 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 f3ff7fe06b13..52c30b7b35d1 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -46,7 +46,7 @@ class ShardNumColumnToFunctionVisitor : public InDepthQueryTreeVisitorWithContex 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 89b014a0360d..9cbf24091e33 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 2736f8052716..56ac3f54b065 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 f7d31a54373b..5c5171d4296d 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 f7d97f71d087..169ef13e732d 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 f821e9c7b42e..90de861651a2 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 d446ca176345..9f133ff075f4 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 e0a0a3313fe8..f5f0fa6f726b 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 3546e3c0e882..2dec0f8257e0 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 4b89dc5fc7ca..a03ad365767d 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 af9d5fd1b34c..ada3252630fb 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 9856164bc9f7..d62e118e1f28 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 035/114] 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 169ef13e732d..198bdbbb71d7 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 90de861651a2..05afc80cba03 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 e0ef21106067..0d67403fa2f2 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 ada3252630fb..34c092c72087 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 8a7eaed07e85648cfb5f99e19a3aa7585a6639e0 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Wed, 3 Jan 2024 09:16:32 -0800 Subject: [PATCH 036/114] Added additional testcases --- .../functions/time-series-functions.md | 2 + src/Functions/seriesDecomposeSTL.cpp | 59 +++++++++++++++++-- .../02813_series_decompose.reference | 2 + .../0_stateless/02813_series_decompose.sql | 8 +++ 4 files changed, 67 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 3f5dea6b4c5e..8491bff9cb05 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -76,6 +76,8 @@ seriesDecomposeSTL(series, period); - `series` - An array of numeric values - `period` - A positive number +The number of data points in `series` should be atleast twice the value of `period`. + **Returned value** - Array of arrays diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index eaa6619aae3e..6b75a8eb87e9 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -152,7 +152,7 @@ class FunctionSeriesDecomposeSTL : public IFunction bool executeNumber( const IColumn & src_data, Float64 period, - ColumnArray::Offset & start, + ColumnArray::Offset & start, ColumnArray::Offset & end, std::vector & seasonal, std::vector & trend, @@ -186,9 +186,9 @@ class FunctionSeriesDecomposeSTL : public IFunction residue = res.remainder; return true; } - catch (const std::exception & e) + catch (...) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, e.what()); + return false; } } }; @@ -196,7 +196,58 @@ REGISTER_FUNCTION(seriesDecomposeSTL) { factory.registerFunction(FunctionDocumentation{ .description = R"( -Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess)", +Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) +Returns an array of three arrays where the first array include seasonal components, the second array - trend, +and the third array - residue component. +https://www.wessa.net/download/stl.pdf + +**Syntax** + +``` sql +seriesDecomposeSTL(series, period); +``` + +**Arguments** + +- `series` - An array of numeric values +- `period` - A positive number + +The number of data points in `series` should be atleast twice the value of `period`. + +**Returned value** + +- Array of arrays + +Type: [Array](../../sql-reference/data-types/array.md). + +**Examples** + +Query: + +``` sql +SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3) AS print_0; +``` + +Result: + +``` text +┌───────────print_0──────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ [[ + -13.529999, -3.1799996, 16.71, -13.53, -3.1799996, 16.71, -13.53, -3.1799996, + 16.71, -13.530001, -3.18, 16.710001, -13.530001, -3.1800003, 16.710001, -13.530001, + -3.1800003, 16.710001, -13.530001, -3.1799994, 16.71, -13.529999, -3.1799994, 16.709997 + ], + [ + 23.63, 23.63, 23.630003, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, + 23.630001, 23.630001, 23.630001, 23.63, 23.630001, 23.630001, 23.63, 23.630001, + 23.630001, 23.63, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, 23.630003 + ], + [ + 0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0, + 0 + ]] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +```)", .categories{"Time series analysis"}}); } } diff --git a/tests/queries/0_stateless/02813_series_decompose.reference b/tests/queries/0_stateless/02813_series_decompose.reference index 110635919fe5..dc30e7f8371f 100644 --- a/tests/queries/0_stateless/02813_series_decompose.reference +++ b/tests/queries/0_stateless/02813_series_decompose.reference @@ -1,2 +1,4 @@ [[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]] [[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7]] +[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]] +[[53.946846,-4.8119445,43.525013,-23.71359,-42.472305,-51.636955,-50.458298,-51.982674,37.62072,-15.9006605,56.65076,-5.809669,57.143845,-2.0370207,54.050922,-4.897961,43.954018,-23.808758,-42.651337,-51.86827,-50.709732,-52.18156,37.734905,-15.853402,56.91643,-5.8815174,57.253094,-2.012879,54.157806,-4.9817176,44.384747,-23.902956,-42.830154,-52.10025,-50.96271,-52.3829,37.84573,-15.81032,57.177113,-5.958963,57.356136,-1.9952412,54.27533,-5.066312,44.878296,-23.956438,-42.993656,-52.337124,-51.208073,-52.615646,37.91102,-15.8062525,57.49891,-6.056076,57.45604,-1.9797823,54.39525,-5.1483474,45.374573],[88.028534,88.95315,89.87776,90.802376,91.64913,92.49588,93.342636,94.19737,95.0521,95.90684,96.712975,97.51912,98.32526,98.36342,98.40158,98.43974,98.36777,98.29579,98.223816,98.536446,98.849075,99.161705,99.7552,100.348694,100.94219,101.53184,102.12149,102.711136,103.79921,104.88729,105.975365,107.50462,109.033875,110.56313,111.79767,113.032196,114.26673,115.02128,115.775826,116.53037,117.15541,117.78044,118.40548,118.86489,119.3243,119.783714,120.04031,120.29691,120.55351,120.78621,121.01891,121.25161,121.533585,121.81555,122.09753,122.41821,122.7389,123.059586,123.39267],[-2.97538,2.8587952,-23.402771,0.91121674,4.8231735,9.141075,8.115662,10.785301,0.32717896,5.99382,-12.363731,5.29055,0.53089905,-2.3264008,-3.4524994,1.4582214,-2.321785,2.51297,5.4275208,3.3318253,5.8606567,0.019859314,-4.4901123,-12.495293,-5.8586197,-1.650322,-11.374588,4.3017426,4.042984,1.094429,9.639885,3.3983307,-3.20372,-5.462883,-5.834961,-6.649292,-1.1124649,3.7890396,16.047066,-2.5714111,8.488449,-2.785202,2.319191,-0.79857635,13.797401,-5.827278,-6.0466614,-5.9597855,-7.3454437,-3.1705627,6.0700684,3.5546417,1.9675064,-0.7594757,2.446434,0.5615692,0.86585236,-3.9112396,1.2327576]] diff --git a/tests/queries/0_stateless/02813_series_decompose.sql b/tests/queries/0_stateless/02813_series_decompose.sql index eae9a84117b9..409499eae246 100644 --- a/tests/queries/0_stateless/02813_series_decompose.sql +++ b/tests/queries/0_stateless/02813_series_decompose.sql @@ -1,8 +1,16 @@ -- Tags: no-fasttest, no-cpu-aarch64 -- Tag no-cpu-aarch64: values generated are slighly different on aarch64 +DROP TABLE IF EXISTS tb2; + +CREATE TABLE tb2 (`n` UInt32, `a` Array(Float64)) ENGINE = MergeTree PRIMARY KEY n; +INSERT INTO tb2 VALUES (3,[10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34]); +INSERT INTO tb2 VALUES (14, [139, 87, 110, 68, 54, 50, 51, 53, 133, 86, 141, 97, 156, 94, 149, 95, 140, 77, 61, 50, 54, 47, 133, 72, 152, 94, 148, 105, 162, 101, 160, 87, 63, 53, 55, 54, 151, 103, 189, 108, 183, 113, 175, 113, 178, 90, 71, 62, 62, 65, 165, 109, 181, 115, 182, 121, 178, 114, 170]); + SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3); SELECT seriesDecomposeSTL([2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], 0); +SELECT seriesDecomposeSTL(a, n) FROM tb2; +DROP TABLE IF EXISTS tb2; SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5); -- { serverError ILLEGAL_COLUMN} SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5.2); --{ serverError ILLEGAL_COLUMN} SELECT seriesDecomposeSTL(); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From 62521d30ce9c07dd3fcba53b958354c480634ef1 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Wed, 3 Jan 2024 09:34:58 -0800 Subject: [PATCH 037/114] fix style-check --- docs/en/sql-reference/functions/time-series-functions.md | 2 +- src/Functions/seriesDecomposeSTL.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 8491bff9cb05..84d5acb098a7 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -76,7 +76,7 @@ seriesDecomposeSTL(series, period); - `series` - An array of numeric values - `period` - A positive number -The number of data points in `series` should be atleast twice the value of `period`. +The number of data points in `series` should be at least twice the value of `period`. **Returned value** diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 6b75a8eb87e9..4145057a2edb 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -212,7 +212,7 @@ seriesDecomposeSTL(series, period); - `series` - An array of numeric values - `period` - A positive number -The number of data points in `series` should be atleast twice the value of `period`. +The number of data points in `series` should be at least twice the value of `period`. **Returned value** From f652c6050cc59a75ab4f1e6e701647761084efc9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 17:35:37 +0000 Subject: [PATCH 038/114] Implement idnaEn/Decode() --- .../functions/string-functions.md | 74 +++++++- src/Functions/punycode.cpp | 177 +++++++++++++++++- .../queries/0_stateless/02932_idna.reference | 75 ++++++++ tests/queries/0_stateless/02932_idna.sql | 123 ++++++++++++ .../0_stateless/02932_punycode.reference | 8 +- tests/queries/0_stateless/02932_punycode.sql | 82 ++++---- 6 files changed, 490 insertions(+), 49 deletions(-) create mode 100644 tests/queries/0_stateless/02932_idna.reference create mode 100644 tests/queries/0_stateless/02932_idna.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index eb5cb29c5027..668592a6fd9e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1385,7 +1385,7 @@ Result: ## punycodeEncode -Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) of a string. +Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) representation of a string. The string must be UTF8-encoded, otherwise an exception is thrown. **Syntax** @@ -1457,6 +1457,78 @@ Result: Like `punycodeDecode` but returns `NULL` in case of an error instead of throwing an exception. +## idnaEncode + +Returns the the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +The string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown. +Note: No percent decoding or trimming of tabs, spaces or control characters is performed. + +**Syntax** + +```sql +idnaEncode(val) +``` + +**Arguments** + +- `val` - Input value. [String](../data-types/string.md) + +**Returned value** + +- A ASCII representation according to the IDNA mechanism of the input value. [String](../data-types/string.md) + +**Example** + +``` sql +select idnaEncode('straße.münchen.de'); +``` + +Result: + +```result +┌─idnaEncode('straße.münchen.de')─────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────────┘ +``` + +## idnaEncodeOrNull + +Like `idnaEncode` but returns `NULL` in case of an error instead of throwing an exception. + +## idnaDecode + +Returns the the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +In case of an error (e.g. because the input is invalid), the input string is returned. +Note that repeated application of `idnaEncode()` and `idnaDecode()` does not necessarily return the original string due to case normalization. + +**Syntax** + +```sql +idnaDecode(val) +``` + +**Arguments** + +- `val` - Input value. [String](../data-types/string.md) + +**Returned value** + +- A Unicode (UTF-8) representation according to the IDNA mechanism of the input value. [String](../data-types/string.md) + +**Example** + +``` sql +select idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de'); +``` + +Result: + +```result +┌─idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de')─┐ +│ straße.münchen.de │ +└───────────────────────────────────────────────┘ +``` + ## byteHammingDistance Calculates the [hamming distance](https://en.wikipedia.org/wiki/Hamming_distance) between two byte strings. diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index 869b79daa10a..d015b6212c81 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -15,6 +15,8 @@ # pragma clang diagnostic ignored "-Wnewline-eof" #endif # include +# include +# include # include #ifdef __clang__ # pragma clang diagnostic pop @@ -38,14 +40,23 @@ enum class ErrorHandling Null }; +/// Implementation of +/// - punycode(En|De)code[OrNull](), see [1] +/// - idna(En|De)code[OrNull](), see [2, 3] +/// +/// [1] https://en.wikipedia.org/wiki/Punycode +/// [2] https://en.wikipedia.org/wiki/Internationalized_domain_name#ToASCII_and_ToUnicode +/// [3] https://www.unicode.org/reports/tr46/#ToUnicode + +/// Kind of similar to FunctionStringToString but accepts String arguments only, and supports `OrNull` overloads template -class FunctionPunycode : public IFunction +class FunctionIdnaPunycodeBase : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared>(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } @@ -73,7 +84,9 @@ class FunctionPunycode : public IFunction if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); if constexpr (Impl::error_handling == ErrorHandling::Null) @@ -198,11 +211,111 @@ struct PunycodeDecodeImpl } }; + +/// Translates a UTF-8 string (typically an Internationalized Domain Name for Applications, IDNA) to an ASCII-encoded equivalent. The +/// encoding is performed per domain component and based on Punycode with ASCII Compatible Encoding (ACE) prefix "xn--". +/// Example: "straße.münchen.de" --> "xn--strae-oqa.xn--mnchen-3ya.de" +/// Note: doesn't do percent decoding. Doesn't trim tabs, spaces or control characters. Expects non-empty inputs. +template +struct IdnaEncodeImpl +{ + static constexpr auto error_handling = error_handling_; + static constexpr auto name = (error_handling == ErrorHandling::Null) ? "idnaEncodeOrNull" : "idnaEncode"; + + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets, + ColumnUInt8::MutablePtr & col_res_null) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + if constexpr (error_handling == ErrorHandling::Null) + col_res_null = ColumnUInt8::create(rows, 0); + + size_t prev_offset = 0; + std::string ascii; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + std::string_view value_view(value, value_length); + + if (!value_view.empty()) /// to_ascii() expects non-empty input + { + ascii = ada::idna::to_ascii(value_view); + const bool ok = !ascii.empty(); + if (!ok) + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); + } + else + { + ascii.clear(); + col_res_null->getData()[row] = 1; + } + } + } + + res_data.insert(ascii.c_str(), ascii.c_str() + ascii.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + ascii.clear(); + } + } +}; + + +/// As per the specification, invalid inputs are returned as is, i.e. there is no special error handling. +/// Therefore, this struct implements no error handling. +struct IdnaDecodeImpl +{ + static constexpr auto error_handling = ErrorHandling::Throw; /// dummy + static constexpr auto name = "idnaDecode"; + + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets, + ColumnUInt8::MutablePtr & /*col_res_null*/) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::string unicode; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + std::string_view value_view(value, value_length); + + unicode = ada::idna::to_unicode(value_view); + + res_data.insert(unicode.c_str(), unicode.c_str() + unicode.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + unicode.clear(); + } + } +}; + + } REGISTER_FUNCTION(Punycode) { - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeEncode(str)", @@ -219,7 +332,7 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Returns NULL in case of error)", .syntax="punycodeEncode(str)", @@ -236,7 +349,7 @@ Computes a Punycode representation of a string. Returns NULL in case of error)", }} }); - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Throws an exception in case of error.)", .syntax="punycodeDecode(str)", @@ -253,7 +366,7 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction>>(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string. Returns NULL in case of error)", .syntax="punycodeDecode(str)", @@ -270,6 +383,56 @@ Computes a Punycode representation of a string. Returns NULL in case of error)", }} }); + factory.registerFunction>>(FunctionDocumentation{ + .description=R"( +Computes an ASCII representation of an Internationalized Domain Name. Throws an exception in case of error.)", + .syntax="idnaEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncode('straße.münchen.de') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction>>(FunctionDocumentation{ + .description=R"( +Computes a ASCII representation of an Internationalized Domain Name. Returns NULL in case of error)", + .syntax="punycodeEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncodeOrNull('München') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction>(FunctionDocumentation{ + .description=R"( +Computes a Unicode representation of an Internationalized Domain Name.)", + .syntax="idnaDecode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de') AS unicode;", + R"( +┌─unicode───────────┐ +│ straße.münchen.de │ +└───────────────────┘ + )" + }} + }); } } diff --git a/tests/queries/0_stateless/02932_idna.reference b/tests/queries/0_stateless/02932_idna.reference new file mode 100644 index 000000000000..8bfba53ad11b --- /dev/null +++ b/tests/queries/0_stateless/02932_idna.reference @@ -0,0 +1,75 @@ +-- Negative tests +-- Regular cases +straße.de xn--strae-oqa.de straße.de xn--strae-oqa.de straße.de +2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 +AMAZON amazon amazon amazon amazon +aa-- aa-- aa-- aa-- aa-- +a†-- xn--a---kp0a a†-- xn--a---kp0a a†-- +ab--c ab--c ab--c ab--c ab--c +-† xn----xhn -† xn----xhn -† +-x.xn--zca -x.xn--zca -x.ß -x.xn--zca -x.ß +x-.xn--zca x-.xn--zca x-.ß x-.xn--zca x-.ß +x-.ß x-.xn--zca x-.ß x-.xn--zca x-.ß +x..ß x..xn--zca x..ß x..xn--zca x..ß +128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 +xn--zca.xn--zca xn--zca.xn--zca ß.ß xn--zca.xn--zca ß.ß +xn--zca.ß xn--zca.xn--zca ß.ß xn--zca.xn--zca ß.ß +x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x +x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß +x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß +01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x +≠ xn--1ch ≠ xn--1ch ≠ +aa-- aa-- aa-- aa-- +ab--c ab--c ab--c ab--c +-x -x -x -x + +xn--1ch ≠ xn--1ch xn--1ch +xn--dqd20apc ᄎᆞᆷ xn--dqd20apc xn--dqd20apc +xn--gdh ≮ xn--gdh xn--gdh +xn--80aaa0ahbbeh4c йайзаакпий xn--80aaa0ahbbeh4c xn--80aaa0ahbbeh4c +xn--3bs854c 团淄 xn--3bs854c xn--3bs854c +xn--mgb9awbf عمان xn--mgb9awbf xn--mgb9awbf +xn--mgbaam7a8h امارات xn--mgbaam7a8h xn--mgbaam7a8h +xn--mgbbh1a71e بھارت xn--mgbbh1a71e xn--mgbbh1a71e +xn--s7y.com 短.com xn--s7y.com xn--s7y.com +xn--55qx5d.xn--tckwe 公司.コム xn--55qx5d.xn--tckwe xn--55qx5d.xn--tckwe +xn--4dbrk0ce ישראל xn--4dbrk0ce xn--4dbrk0ce +xn--zckzah テスト xn--zckzah xn--zckzah +xn--p1ai.com рф.com xn--p1ai.com xn--p1ai.com +xn--mxahbxey0c.gr εχαμπλε.gr xn--mxahbxey0c.gr xn--mxahbxey0c.gr +xn--h2brj9c भारत xn--h2brj9c xn--h2brj9c +xn--d1acpjx3f.xn--p1ai яндекс.рф xn--d1acpjx3f.xn--p1ai xn--d1acpjx3f.xn--p1ai +xn--q9jyb4c みんな xn--q9jyb4c xn--q9jyb4c +xn--sterreich-z7a.at österreich.at xn--sterreich-z7a.at xn--sterreich-z7a.at +xn--h2breg3eve.xn--h2brj9c भारतम्.भारत xn--h2breg3eve.xn--h2brj9c xn--h2breg3eve.xn--h2brj9c +ejemplo.xn--q9jyb4c ejemplo.みんな ejemplo.xn--q9jyb4c ejemplo.xn--q9jyb4c +xn--9t4b11yi5a.com 테스트.com xn--9t4b11yi5a.com xn--9t4b11yi5a.com +xn--gk3at1e.com 通販.com xn--gk3at1e.com xn--gk3at1e.com +xn--42c2d9a คอม xn--42c2d9a xn--42c2d9a +1xn-- 1xn-- 1xn-- 1xn-- +xn--bih.com ⌘.com xn--bih.com xn--bih.com +xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c موقع.وزارة-الأتصالات.مصر xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c +xn--mgbb9fbpob موبايلي xn--mgbb9fbpob xn--mgbb9fbpob +xn--55qw42g.xn--55qw42g 公益.公益 xn--55qw42g.xn--55qw42g xn--55qw42g.xn--55qw42g +≠ ≠ xn--1ch xn--1ch +ファッション.biz ファッション.biz xn--bck1b9a5dre4c.biz xn--bck1b9a5dre4c.biz +-- Special cases + + + +\N +\N +\N +\N +\N +\N +\N + +münchen xn--mnchen-3ya münchen xn--mnchen-3ya münchen +straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de +london.co.uk london.co.uk +microsoft.com microsoft.com +straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de +xn-- \N +xn-- \N +xn--tešla \N diff --git a/tests/queries/0_stateless/02932_idna.sql b/tests/queries/0_stateless/02932_idna.sql new file mode 100644 index 000000000000..3572d4a6aeca --- /dev/null +++ b/tests/queries/0_stateless/02932_idna.sql @@ -0,0 +1,123 @@ +-- Tags: no-fasttest +-- no-fasttest: requires idna library + +-- See also 02932_punycode.sql + +SELECT '-- Negative tests'; + +SELECT idnaEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaEncodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT idnaEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaEncodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT idnaEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaEncodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT idnaEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaEncodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT '-- Regular cases'; + +-- The test cases originate from the ada idna unit tests: +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_alternating.txt +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_unicode_alternating.txt + +SELECT 'straße.de' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '2001:4860:4860::8888' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'AMAZON' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'aa--' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'a†--' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'ab--c' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '-†' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '-x.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x-.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x-.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x..ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '128.0,0.1' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'xn--zca.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'xn--zca.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; +SELECT '≠' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; + +SELECT 'aa--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'ab--c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT '-x' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT '' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--1ch' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--dqd20apc' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--gdh' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--80aaa0ahbbeh4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--3bs854c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--mgb9awbf' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--mgbaam7a8h' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--mgbbh1a71e' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--s7y.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--55qx5d.xn--tckwe' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--4dbrk0ce' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--zckzah' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--p1ai.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--mxahbxey0c.gr' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--d1acpjx3f.xn--p1ai' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--sterreich-z7a.at' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--h2breg3eve.xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'ejemplo.xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--9t4b11yi5a.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--gk3at1e.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--42c2d9a' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT '1xn--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--bih.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--mgbb9fbpob' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'xn--55qw42g.xn--55qw42g' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT '≠' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; +SELECT 'ファッション.biz' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; + +SELECT '-- Special cases'; + +SELECT idnaEncode(''); +SELECT idnaEncodeOrNull(''); +SELECT idnaDecode(''); + +SELECT idnaEncode(NULL); +SELECT idnaEncodeOrNull(NULL); +SELECT idnaDecode(NULL); + +-- garbage IDNA/unicode values, see +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_invalid.txt +-- only idnaEncode() is tested, idnaDecode() has by definition no invalid input values +SELECT idnaEncode('xn--'); -- { serverError BAD_ARGUMENTS } +SELECT idnaEncodeOrNull('xn--'); +SELECT idnaEncode('ﻱa'); -- { serverError BAD_ARGUMENTS } +SELECT idnaEncodeOrNull('ﻱa'); +SELECT idnaEncode('xn--a-yoc'); -- { serverError BAD_ARGUMENTS } +SELECT idnaEncodeOrNull('xn--a-yoc'); +SELECT idnaEncode('xn--tešla'); -- { serverError BAD_ARGUMENTS } +SELECT idnaEncodeOrNull('xn--tešla'); + +-- long input +-- SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS idna, idnaEncode(idna) AS ascii, idnaEncodeOrNull(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull FORMAT Vertical; + +-- non-const values +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (idna String) ENGINE=MergeTree ORDER BY idna; +INSERT INTO tab VALUES ('straße.münchen.de') ('') ('münchen'); +SELECT idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull FROM tab; +DROP TABLE tab; + +-- non-const values with a few invalid values for testing the OrNull variants +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (ascii String) ENGINE=MergeTree ORDER BY ascii; +INSERT INTO tab VALUES ('xn--') ('london.co.uk') ('straße.münchen.de') ('xn--tešla') ('microsoft.com') ('xn--'); +SELECT ascii, idnaEncode(ascii) AS original FROM tab; -- { serverError BAD_ARGUMENTS } +SELECT ascii, idnaEncodeOrNull(ascii) AS original FROM tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02932_punycode.reference b/tests/queries/0_stateless/02932_punycode.reference index a722894bce8e..76508525b199 100644 --- a/tests/queries/0_stateless/02932_punycode.reference +++ b/tests/queries/0_stateless/02932_punycode.reference @@ -34,7 +34,13 @@ MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 MajiK \N \N \N -Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +Row 1: +────── +str: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +puny: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa +original: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +punyOrNull: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa +originalOrNull: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. München Mnchen-3ya München Mnchen-3ya München abc abc- abc abc- abc aäoöuü aou-qla5gqb aäoöuü aou-qla5gqb aäoöuü diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql index a142848d427b..8df47cbf3da0 100644 --- a/tests/queries/0_stateless/02932_punycode.sql +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -1,82 +1,84 @@ -- Tags: no-fasttest -- no-fasttest: requires idna library +-- See also 02932_idna.sql + SELECT '-- Negative tests'; -SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '-- Regular cases'; -- The test cases originate from the ada idna unit tests: ---- https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt - -SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; -SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt + +SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; +SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; SELECT '-- Special cases'; -SELECT punycodeDecode(''); -SELECT punycodeDecodeOrNull(''); SELECT punycodeEncode(''); SELECT punycodeEncodeOrNull(''); +SELECT punycodeDecode(''); +SELECT punycodeDecodeOrNull(''); -SELECT punycodeDecode(NULL); -SELECT punycodeDecodeOrNull(NULL); SELECT punycodeEncode(NULL); SELECT punycodeEncodeOrNull(NULL); +SELECT punycodeDecode(NULL); +SELECT punycodeDecodeOrNull(NULL); -- garbage Punycode-encoded values SELECT punycodeDecode('no punycode'); -- { serverError BAD_ARGUMENTS } SELECT punycodeDecodeOrNull('no punycode'); -- long input -SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull FORMAT Vertical; -- non-const values DROP TABLE IF EXISTS tab; CREATE TABLE tab (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO tab VALUES ('abc') ('aäoöuü') ('München'); -SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) as originalOrNull FROM tab; +SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull FROM tab; DROP TABLE tab; -- non-const values with a few invalid values for testing the OrNull variants From 48a3968d5a7d955c3acbbecea963d2b60a20f076 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jan 2024 20:06:36 +0000 Subject: [PATCH 039/114] 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 198bdbbb71d7..f0cf52aa46d5 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 a813431fcb58a50650e7e9eef29d41dd33a6397f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jan 2024 20:09:12 +0000 Subject: [PATCH 040/114] Fix spelling --- .../check-style/aspell-ignore/en/aspell-dict.txt | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f339f4518784..11fab98cf2df 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -344,6 +344,7 @@ Hypot IANA IDE IDEs +IDNA IMDS INFILE INSERTed @@ -701,8 +702,6 @@ PrettySpaceMonoBlock PrettySpaceNoEscapes PrettySpaceNoEscapesMonoBlock Prewhere -TotalPrimaryKeyBytesInMemory -TotalPrimaryKeyBytesInMemoryAllocated PrivateKeyPassphraseHandler ProfileEvents Profiler @@ -913,6 +912,7 @@ ThreadsInOvercommitTracker Timeunit TinyLog Tkachenko +ToASCII ToCenterChild ToChildren ToGeo @@ -921,10 +921,13 @@ ToIPv ToParent ToSnowflake ToString +ToUnicode Toolset TopK TotalBytesOfMergeTreeTables TotalPartsOfMergeTreeTables +TotalPrimaryKeyBytesInMemory +TotalPrimaryKeyBytesInMemoryAllocated TotalRowsOfMergeTreeTables TotalTemporaryFiles Tradeoff @@ -1652,6 +1655,9 @@ hyvor icosahedron icudata idempotency +idnaDecode +idnaEncode +idnaEncodeOrNull ifNotFinite ifNull iframe @@ -1849,14 +1855,14 @@ metrica metroHash mfedotov minMap +minSampleSizeContinuous +minSampleSizeConversion mindsdb minimalistic mininum miniselect minmap minmax -minSampleSizeContinuous -minSampleSizeConversion mins misconfiguration mispredictions @@ -2077,7 +2083,9 @@ pseudorandomize psql ptrs punycodeDecode +punycodeDecodeOrNull punycodeEncode +punycodeEncodeOrNull pushdown pwrite py From c2ebb9a5d9b62dd988031be10494627fd03e0864 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Wed, 3 Jan 2024 13:05:52 -0800 Subject: [PATCH 041/114] fix for const column input --- src/Functions/seriesDecomposeSTL.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 4145057a2edb..b6303f8cda6b 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -66,6 +66,17 @@ class FunctionSeriesDecomposeSTL : public IFunction { ColumnPtr array_ptr = arguments[0].column; const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + if (!array) + { + const ColumnConst * const_array = checkAndGetColumnConst( + arguments[0].column.get()); + if (!const_array) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), getName()); + + array_ptr = const_array->convertToFullColumn(); + array = assert_cast(array_ptr.get()); + } const IColumn & src_data = array->getData(); const ColumnArray::Offsets & src_offsets = array->getOffsets(); @@ -168,7 +179,7 @@ class FunctionSeriesDecomposeSTL : public IFunction size_t len = end - start; if (len < 4) throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName()); - else if (period > (len / 2)) + else if (period > (len / 2.0)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The series should have data of at least two period lengths for function {}", getName()); From 85b4d6c838bde329f34d74feb3d2ffde1de9f1a7 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 4 Jan 2024 01:46:03 +0000 Subject: [PATCH 042/114] fix shell check Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index a88c0ddd5e90..eab44e74d887 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -15,7 +15,7 @@ chmod 777 ${FILE_PATH} FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${FILE_PATH}/${FILE} -symlink_path_with_regex=${FILE_PATH}*/${FILE} +symlink_path_with_regex="${FILE_PATH}*/${FILE}" file_path=$CUR_DIR/${FILE} touch ${file_path} From 8f26e2af67614984d1e7db0330f556e1fb584c18 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Jan 2024 13:52:08 +0000 Subject: [PATCH 043/114] Even better Keeper startup --- src/Coordination/KeeperServer.cpp | 98 ++++++++++++++----------------- 1 file changed, 44 insertions(+), 54 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fb56d58cb723..965e743da393 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -4,6 +4,7 @@ #include "config.h" #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -196,13 +198,9 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server nuraft::raft_server::commit_in_bg(); } - void commitLogs(uint64_t index_to_commit, bool initial_commit_exec) + std::unique_lock lockRaft() { - leader_commit_index_.store(index_to_commit); - quick_commit_index_ = index_to_commit; - lagging_sm_target_index_ = index_to_commit; - - commit_in_bg_exec(0, initial_commit_exec); + return std::unique_lock(lock_); } using nuraft::raft_server::raft_server; @@ -518,6 +516,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) { std::vector> entries; + entries.reserve(requests_for_sessions.size()); for (const auto & request_for_session : requests_for_sessions) entries.push_back(getZooKeeperLogEntry(request_for_session)); @@ -630,32 +629,32 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ { const auto preprocess_logs = [&] { + auto lock = raft_instance->lockRaft(); keeper_context->local_logs_preprocessed = true; auto log_store = state_manager->load_log_store(); - if (last_log_idx_on_disk > 0 && last_log_idx_on_disk > state_machine->last_commit_index()) + auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, log_store->next_slot()); + + if (log_entries->empty()) { - auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, last_log_idx_on_disk + 1); + LOG_INFO(log, "All local log entries preprocessed"); + return; + } - size_t preprocessed = 0; - LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); - auto idx = state_machine->last_commit_index() + 1; - for (const auto & entry : *log_entries) - { - if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) - state_machine->pre_commit(idx, entry->get_buf()); + size_t preprocessed = 0; + LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); + auto idx = state_machine->last_commit_index() + 1; + for (const auto & entry : *log_entries) + { + if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) + state_machine->pre_commit(idx, entry->get_buf()); - ++idx; - ++preprocessed; + ++idx; + ++preprocessed; - if (preprocessed % 50000 == 0) - LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); - } - LOG_INFO(log, "Preprocessing done"); - } - else - { - LOG_INFO(log, "All local log entries preprocessed"); + if (preprocessed % 50000 == 0) + LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); } + LOG_INFO(log, "Preprocessing done"); }; switch (type) @@ -666,43 +665,34 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ /// until we preprocess all stored logs return nuraft::cb_func::ReturnCode::ReturnNull; } - case nuraft::cb_func::InitialBatchCommited: - { - preprocess_logs(); - break; - } case nuraft::cb_func::GotAppendEntryReqFromLeader: { + /// maybe we got snapshot installed + if (state_machine->last_commit_index() >= last_log_idx_on_disk) + { + preprocess_logs(); + break; + } + auto & req = *static_cast(param->ctx); - if (req.get_commit_idx() == 0 || req.log_entries().empty()) + if (req.log_entries().empty()) break; - auto last_committed_index = state_machine->last_commit_index(); - // Actual log number. - auto index_to_commit = std::min({last_log_idx_on_disk, req.get_last_log_idx(), req.get_commit_idx()}); + if (req.get_last_log_idx() < last_log_idx_on_disk) + last_log_idx_on_disk = req.get_last_log_idx(); + /// we don't want to accept too many new logs before we preprocess all the local logs + /// because the next log index is decreased on each failure we need to also accept requests when it's near last_log_idx_on_disk + /// so the counter is reset on the leader side + else if (raft_instance->get_target_committed_log_idx() >= last_log_idx_on_disk && req.get_last_log_idx() > last_log_idx_on_disk) + return nuraft::cb_func::ReturnNull; - if (index_to_commit > last_committed_index) - { - LOG_TRACE(log, "Trying to commit local log entries, committing upto {}", index_to_commit); - raft_instance->commitLogs(index_to_commit, true); - /// after we manually committed all the local logs we can, we assert that all of the local logs are either - /// committed or preprocessed - if (!keeper_context->local_logs_preprocessed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Local logs are not preprocessed"); - } - else if (last_log_idx_on_disk <= last_committed_index) - { - keeper_context->local_logs_preprocessed = true; - } - else if - ( - index_to_commit == 0 || - (index_to_commit == last_committed_index && last_log_idx_on_disk > index_to_commit) /// we need to rollback all the logs so we preprocess all of them - ) - { + break; + } + case nuraft::cb_func::StateMachineExecution: + { + if (state_machine->last_commit_index() >= last_log_idx_on_disk) preprocess_logs(); - } break; } default: From 13749f550f85a9222c41fabe1ede19b8347f6380 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Jan 2024 07:59:44 +0000 Subject: [PATCH 044/114] Lower log levels for some Raft logs --- contrib/NuRaft | 2 +- src/Coordination/LoggerWrapper.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index b7ea89b817a1..636e83c33b22 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit b7ea89b817a18dc0eafc1f909d568869f02d2d04 +Subproject commit 636e83c33b2243d88935d8bf78022f225f315154 diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index ae3ff1553b09..d092a8d44407 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -13,6 +13,7 @@ class LoggerWrapper : public nuraft::logger static inline const std::unordered_map LEVELS = { + {LogsLevel::test, Poco::Message::Priority::PRIO_TEST}, {LogsLevel::trace, Poco::Message::Priority::PRIO_TRACE}, {LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG}, {LogsLevel::information, Poco::Message::PRIO_INFORMATION}, From 2e14cfb526870955e849da754a42891b82f02c6a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 03:40:26 +0000 Subject: [PATCH 045/114] 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 4e057861f606..680b80d92f7e 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 13e1adbb7024..fc415b73ec1d 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 c142531cd05f..511704059ecf 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 949d65926b37..20bdeef79187 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 a66d6085a747..8c129b1bfbba 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.h +++ b/src/IO/ZstdDeflatingWriteBuffer.h @@ -17,6 +17,7 @@ class ZstdDeflatingWriteBuffer : public WriteBufferWithOwnMemoryDecorator 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 63804d2d86f2..23a1a703b162 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 6d8f1ab55cb1..02ca2734ff85 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 1a12c09a8c7b..c32da278e496 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 fdbb5e9f1719..a22c5903420d 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -723,13 +723,13 @@ class HDFSSink : public SinkToStorage 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 9564bad485c2..0e00c62111c3 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -533,7 +533,12 @@ class StorageAzureBlobSink : public SinkToStorage , 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 25bb6691ff68..0c8394064629 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1485,8 +1485,12 @@ class StorageFileSink final : public SinkToStorage /// 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 60ae7f219f43..1af0b638df95 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -932,6 +932,7 @@ class StorageS3Sink : public SinkToStorage blob_log->query_id = context->getCurrentQueryId(); } + const auto & settings = context->getSettingsRef(); write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( configuration_.client, @@ -944,7 +945,8 @@ class StorageS3Sink : public SinkToStorage 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 d6b6f5af61c7..39fd6195698e 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 0adac742f816d8e1557317a8b2006c500e2198e9 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Thu, 4 Jan 2024 07:07:08 -0800 Subject: [PATCH 046/114] fix testcase --- tests/queries/0_stateless/02813_series_decompose.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02813_series_decompose.sql b/tests/queries/0_stateless/02813_series_decompose.sql index 409499eae246..abaa99b38171 100644 --- a/tests/queries/0_stateless/02813_series_decompose.sql +++ b/tests/queries/0_stateless/02813_series_decompose.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS tb2; -CREATE TABLE tb2 (`n` UInt32, `a` Array(Float64)) ENGINE = MergeTree PRIMARY KEY n; +CREATE TABLE tb2 (`n` UInt32, `a` Array(Float64)) ENGINE = Memory; INSERT INTO tb2 VALUES (3,[10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34]); INSERT INTO tb2 VALUES (14, [139, 87, 110, 68, 54, 50, 51, 53, 133, 86, 141, 97, 156, 94, 149, 95, 140, 77, 61, 50, 54, 47, 133, 72, 152, 94, 148, 105, 162, 101, 160, 87, 63, 53, 55, 54, 151, 103, 189, 108, 183, 113, 175, 113, 178, 90, 71, 62, 62, 65, 165, 109, 181, 115, 182, 121, 178, 114, 170]); From 03951e1b21469e747f2a894aff5365c8b30565eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Jan 2024 21:59:32 +0100 Subject: [PATCH 047/114] 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 4c9d45423ee7..72b10d768f1d 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 799ab91d3326..1a4652df23a1 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 048/114] 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 d6e11fb96136..4588f68cacd1 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 049/114] 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 20bdeef79187..3b474a4de745 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 000000000000..749fce669df1 --- /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 000000000000..fc67bca58160 --- /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 050/114] 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 6e087467bb93..1028edfffd09 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 680b80d92f7e..db0ffebe2a7b 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 051/114] 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 1028edfffd09..0cf6dbd80a16 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 db0ffebe2a7b..254f7338a308 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 e2944890f2ed887d99b9018eaf35c01f2724d154 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Jan 2024 10:28:08 +0000 Subject: [PATCH 052/114] Use master commit --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 636e83c33b22..2f5f52c4d8c8 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 636e83c33b2243d88935d8bf78022f225f315154 +Subproject commit 2f5f52c4d8c87c2a3a3d101ca3a0194c9b77526f From 2cc2a9089c9d06bb5f7cc6b8e2bbedc11bac2fc1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 5 Jan 2024 12:18:21 +0000 Subject: [PATCH 053/114] Incorporate review feedback --- .../functions/string-functions.md | 18 +- src/Functions/idna.cpp | 202 ++++++++++++ src/Functions/punycode.cpp | 308 +++--------------- .../queries/0_stateless/02932_idna.reference | 61 ++-- tests/queries/0_stateless/02932_idna.sql | 159 ++++----- .../0_stateless/02932_punycode.reference | 79 ++--- tests/queries/0_stateless/02932_punycode.sql | 90 +++-- 7 files changed, 447 insertions(+), 470 deletions(-) create mode 100644 src/Functions/idna.cpp diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 668592a6fd9e..3461ab28bf99 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1386,7 +1386,7 @@ Result: ## punycodeEncode Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) representation of a string. -The string must be UTF8-encoded, otherwise an exception is thrown. +The string must be UTF8-encoded, otherwise the behavior is undefined. **Syntax** @@ -1416,10 +1416,6 @@ Result: └───────────────────────────┘ ``` -## punycodeEncodeOrNull - -Like `punycodeEncode` but returns `NULL` in case of an error instead of throwing an exception. - ## punycodeDecode Returns the UTF8-encoded plaintext of a [Punycode](https://en.wikipedia.org/wiki/Punycode)-encoded string. @@ -1448,19 +1444,19 @@ select punycodeDecode('Mnchen-3ya'); Result: ```result -┌─punycodeEncode('Mnchen-3ya')─┐ +┌─punycodeDecode('Mnchen-3ya')─┐ │ München │ └──────────────────────────────┘ ``` -## punycodeDecodeOrNull +## tryPunycodeDecode -Like `punycodeDecode` but returns `NULL` in case of an error instead of throwing an exception. +Like `punycodeDecode` but returns an empty string if no valid Punycode-encoded string is given. ## idnaEncode Returns the the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. -The string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown. +The input string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown. Note: No percent decoding or trimming of tabs, spaces or control characters is performed. **Syntax** @@ -1491,9 +1487,9 @@ Result: └─────────────────────────────────────┘ ``` -## idnaEncodeOrNull +## tryIdnaEncode -Like `idnaEncode` but returns `NULL` in case of an error instead of throwing an exception. +Like `idnaEncode` but returns an empty string in case of an error instead of throwing an exception. ## idnaDecode diff --git a/src/Functions/idna.cpp b/src/Functions/idna.cpp new file mode 100644 index 000000000000..087131021c9a --- /dev/null +++ b/src/Functions/idna.cpp @@ -0,0 +1,202 @@ +#include "config.h" + +#if USE_IDNA + +#include +#include +#include + +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wnewline-eof" +#endif +# include +# include +# include +#ifdef __clang__ +# pragma clang diagnostic pop +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; +} + +/// Implementation of +/// - idnaEncode(), tryIdnaEncode() and idnaDecode(), see https://en.wikipedia.org/wiki/Internationalized_domain_name#ToASCII_and_ToUnicode +/// and [3] https://www.unicode.org/reports/tr46/#ToUnicode + + +enum class ErrorHandling +{ + Throw, /// Throw exception + Empty /// Return empty string +}; + + +/// Translates a UTF-8 string (typically an Internationalized Domain Name for Applications, IDNA) to an ASCII-encoded equivalent. The +/// encoding is performed per domain component and based on Punycode with ASCII Compatible Encoding (ACE) prefix "xn--". +/// Example: "straße.münchen.de" --> "xn--strae-oqa.xn--mnchen-3ya.de" +/// Note: doesn't do percent decoding. Doesn't trim tabs, spaces or control characters. Expects non-empty inputs. +template +struct IdnaEncode +{ + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::string ascii; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + + std::string_view value_view(value, value_length); + if (!value_view.empty()) /// to_ascii() expects non-empty input + { + ascii = ada::idna::to_ascii(value_view); + const bool ok = !ascii.empty(); + if (!ok) + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); + } + else + { + static_assert(error_handling == ErrorHandling::Empty); + ascii.clear(); + } + } + } + + res_data.insert(ascii.c_str(), ascii.c_str() + ascii.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + ascii.clear(); + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); + } +}; + +struct IdnaDecode +{ + /// As per the specification, invalid inputs are returned as is, i.e. there is no special error handling. + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::string unicode; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + std::string_view value_view(value, value_length); + + unicode = ada::idna::to_unicode(value_view); + + res_data.insert(unicode.c_str(), unicode.c_str() + unicode.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + unicode.clear(); + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); + } +}; + +struct NameIdnaEncode { static constexpr auto name = "idnaEncode"; }; +struct NameTryIdnaEncode { static constexpr auto name = "tryIdnaEncode"; }; +struct NameIdnaDecode { static constexpr auto name = "idnaDecode"; }; + +using FunctionIdnaEncode = FunctionStringToString, NameIdnaEncode>; +using FunctionTryIdnaEncode = FunctionStringToString, NameTryIdnaEncode>; +using FunctionIdnaDecode = FunctionStringToString; + +REGISTER_FUNCTION(Idna) +{ + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes an ASCII representation of an Internationalized Domain Name. Throws an exception in case of error.)", + .syntax="idnaEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncode('straße.münchen.de') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes a ASCII representation of an Internationalized Domain Name. Returns an empty string in case of error)", + .syntax="punycodeEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncodeOrNull('München') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes a Unicode representation of an Internationalized Domain Name.)", + .syntax="idnaDecode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de') AS unicode;", + R"( +┌─unicode───────────┐ +│ straße.münchen.de │ +└───────────────────┘ + )" + }} + }); +} + +} + +#endif + diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index d015b6212c81..66e5ac6cb7b9 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -2,21 +2,15 @@ #if USE_IDNA -#include #include -#include -#include #include -#include -#include +#include #ifdef __clang__ # pragma clang diagnostic push # pragma clang diagnostic ignored "-Wnewline-eof" #endif # include -# include -# include # include #ifdef __clang__ # pragma clang diagnostic pop @@ -28,93 +22,31 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; } -namespace -{ - -enum class ErrorHandling -{ - Throw, - Null -}; - /// Implementation of -/// - punycode(En|De)code[OrNull](), see [1] -/// - idna(En|De)code[OrNull](), see [2, 3] -/// -/// [1] https://en.wikipedia.org/wiki/Punycode -/// [2] https://en.wikipedia.org/wiki/Internationalized_domain_name#ToASCII_and_ToUnicode -/// [3] https://www.unicode.org/reports/tr46/#ToUnicode +/// - punycodeEncode(), punycodeDecode() and tryPunycodeDecode(), see https://en.wikipedia.org/wiki/Punycode - -/// Kind of similar to FunctionStringToString but accepts String arguments only, and supports `OrNull` overloads -template -class FunctionIdnaPunycodeBase : public IFunction +enum class ErrorHandling { -public: - static constexpr auto name = Impl::name; - - static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared>(); } - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"str", &isString, nullptr, "String"}, - }; - validateFunctionArgumentTypes(*this, arguments, args); - - auto return_type = std::make_shared(); - - if constexpr (Impl::error_handling == ErrorHandling::Null) - return makeNullable(return_type); - else - return return_type; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - auto col_res = ColumnString::create(); - ColumnUInt8::MutablePtr col_res_null; - if (const ColumnString * col = checkAndGetColumn(arguments[0].column.get())) - Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), col_res_null); - else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of argument of function {}", - arguments[0].column->getName(), getName()); - - if constexpr (Impl::error_handling == ErrorHandling::Null) - return ColumnNullable::create(std::move(col_res), std::move(col_res_null)); - else - return col_res; - } + Throw, /// Throw exception + Empty /// Return empty string }; -template -struct PunycodeEncodeImpl +struct PunycodeEncode { - static constexpr auto error_handling = error_handling_; - static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeEncodeOrNull" : "punycodeEncode"; - + /// Encoding-as-punycode can only fail if the input isn't valid UTF8. In that case, returnn undefined output, i.e. garbage-in, garbage-out. static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets, - ColumnUInt8::MutablePtr & col_res_null) + ColumnString::Offsets & res_offsets) { const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII res_offsets.reserve(rows); - if constexpr (error_handling == ErrorHandling::Null) - col_res_null = ColumnUInt8::create(rows, 0); size_t prev_offset = 0; std::u32string value_utf32; @@ -126,21 +58,13 @@ struct PunycodeEncodeImpl const size_t value_utf32_length = ada::idna::utf32_length_from_utf8(value, value_length); value_utf32.resize(value_utf32_length); - ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); + const size_t codepoints = ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); + if (codepoints == 0) + value_utf32.clear(); /// input was empty or it is not valid UTF-8 const bool ok = ada::idna::utf32_to_punycode(value_utf32, value_puny); if (!ok) - { - if constexpr (error_handling == ErrorHandling::Throw) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); - } - else - { - value_puny.clear(); - col_res_null->getData()[row] = 1; - } - } + value_puny.clear(); res_data.insert(value_puny.c_str(), value_puny.c_str() + value_puny.size() + 1); res_offsets.push_back(res_data.size()); @@ -151,27 +75,26 @@ struct PunycodeEncodeImpl value_puny.clear(); /// utf32_to_punycode() appends to its output string } } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); + } }; -template -struct PunycodeDecodeImpl +template +struct PunycodeDecode { - static constexpr auto error_handling = error_handling_; - static constexpr auto name = (error_handling == ErrorHandling::Null) ? "punycodeDecodeOrNull" : "punycodeDecode"; - static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets, - ColumnUInt8::MutablePtr & col_res_null) + ColumnString::Offsets & res_offsets) { const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII res_offsets.reserve(rows); - if constexpr (error_handling == ErrorHandling::Null) - col_res_null = ColumnUInt8::create(rows, 0); size_t prev_offset = 0; std::u32string value_utf32; @@ -191,8 +114,8 @@ struct PunycodeDecodeImpl } else { + static_assert(error_handling == ErrorHandling::Empty); value_utf32.clear(); - col_res_null->getData()[row] = 1; } } @@ -209,115 +132,26 @@ struct PunycodeDecodeImpl value_utf8.clear(); } } -}; - - -/// Translates a UTF-8 string (typically an Internationalized Domain Name for Applications, IDNA) to an ASCII-encoded equivalent. The -/// encoding is performed per domain component and based on Punycode with ASCII Compatible Encoding (ACE) prefix "xn--". -/// Example: "straße.münchen.de" --> "xn--strae-oqa.xn--mnchen-3ya.de" -/// Note: doesn't do percent decoding. Doesn't trim tabs, spaces or control characters. Expects non-empty inputs. -template -struct IdnaEncodeImpl -{ - static constexpr auto error_handling = error_handling_; - static constexpr auto name = (error_handling == ErrorHandling::Null) ? "idnaEncodeOrNull" : "idnaEncode"; - - static void vector( - const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets, - ColumnUInt8::MutablePtr & col_res_null) - { - const size_t rows = offsets.size(); - res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); - if constexpr (error_handling == ErrorHandling::Null) - col_res_null = ColumnUInt8::create(rows, 0); - - size_t prev_offset = 0; - std::string ascii; - for (size_t row = 0; row < rows; ++row) - { - const char * value = reinterpret_cast(&data[prev_offset]); - const size_t value_length = offsets[row] - prev_offset - 1; - std::string_view value_view(value, value_length); - - if (!value_view.empty()) /// to_ascii() expects non-empty input - { - ascii = ada::idna::to_ascii(value_view); - const bool ok = !ascii.empty(); - if (!ok) - { - if constexpr (error_handling == ErrorHandling::Throw) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); - } - else - { - ascii.clear(); - col_res_null->getData()[row] = 1; - } - } - } - - res_data.insert(ascii.c_str(), ascii.c_str() + ascii.size() + 1); - res_offsets.push_back(res_data.size()); - - prev_offset = offsets[row]; - - ascii.clear(); - } - } -}; - - -/// As per the specification, invalid inputs are returned as is, i.e. there is no special error handling. -/// Therefore, this struct implements no error handling. -struct IdnaDecodeImpl -{ - static constexpr auto error_handling = ErrorHandling::Throw; /// dummy - static constexpr auto name = "idnaDecode"; - static void vector( - const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets, - ColumnUInt8::MutablePtr & /*col_res_null*/) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - const size_t rows = offsets.size(); - res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); - - size_t prev_offset = 0; - std::string unicode; - for (size_t row = 0; row < rows; ++row) - { - const char * value = reinterpret_cast(&data[prev_offset]); - const size_t value_length = offsets[row] - prev_offset - 1; - std::string_view value_view(value, value_length); - - unicode = ada::idna::to_unicode(value_view); - - res_data.insert(unicode.c_str(), unicode.c_str() + unicode.size() + 1); - res_offsets.push_back(res_data.size()); - - prev_offset = offsets[row]; - - unicode.clear(); - } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } }; +struct NamePunycodeEncode { static constexpr auto name = "punycodeEncode"; }; +struct NamePunycodeDecode { static constexpr auto name = "punycodeDecode"; }; +struct NameTryPunycodeDecode { static constexpr auto name = "tryPunycodeDecode"; }; -} +using FunctionPunycodeEncode = FunctionStringToString; +using FunctionPunycodeDecode = FunctionStringToString, NamePunycodeDecode>; +using FunctionTryPunycodeDecode = FunctionStringToString, NameTryPunycodeDecode>; REGISTER_FUNCTION(Punycode) { - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string. Throws an exception in case of error.)", +Computes a Punycode representation of a string.)", .syntax="punycodeEncode(str)", .arguments={{"str", "Input string"}}, .returned_value="The punycode representation [String](/docs/en/sql-reference/data-types/string.md).", @@ -332,26 +166,9 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>>(FunctionDocumentation{ - .description=R"( -Computes a Punycode representation of a string. Returns NULL in case of error)", - .syntax="punycodeEncode(str)", - .arguments={{"str", "Input string"}}, - .returned_value="The punycode representation [String](/docs/en/sql-reference/data-types/string.md).", - .examples={ - {"simple", - "SELECT punycodeEncodeOrNull('München') AS puny;", - R"( -┌─puny───────┐ -│ Mnchen-3ya │ -└────────────┘ - )" - }} - }); - - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string. Throws an exception in case of error.)", +Computes a Punycode representation of a string. Throws an exception if the input is not valid Punycode.)", .syntax="punycodeDecode(str)", .arguments={{"str", "A Punycode-encoded string"}}, .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", @@ -366,15 +183,15 @@ Computes a Punycode representation of a string. Throws an exception in case of e }} }); - factory.registerFunction>>(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string. Returns NULL in case of error)", +Computes a Punycode representation of a string. Returns an empty string if the input is not valid Punycode.)", .syntax="punycodeDecode(str)", .arguments={{"str", "A Punycode-encoded string"}}, .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", .examples={ {"simple", - "SELECT punycodeDecodeOrNull('Mnchen-3ya') AS plain;", + "SELECT tryPunycodeDecode('Mnchen-3ya') AS plain;", R"( ┌─plain───┐ │ München │ @@ -382,57 +199,6 @@ Computes a Punycode representation of a string. Returns NULL in case of error)", )" }} }); - - factory.registerFunction>>(FunctionDocumentation{ - .description=R"( -Computes an ASCII representation of an Internationalized Domain Name. Throws an exception in case of error.)", - .syntax="idnaEncode(str)", - .arguments={{"str", "Input string"}}, - .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", - .examples={ - {"simple", - "SELECT idnaEncode('straße.münchen.de') AS ascii;", - R"( -┌─ascii───────────────────────────┐ -│ xn--strae-oqa.xn--mnchen-3ya.de │ -└─────────────────────────────────┘ - )" - }} - }); - - factory.registerFunction>>(FunctionDocumentation{ - .description=R"( -Computes a ASCII representation of an Internationalized Domain Name. Returns NULL in case of error)", - .syntax="punycodeEncode(str)", - .arguments={{"str", "Input string"}}, - .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", - .examples={ - {"simple", - "SELECT idnaEncodeOrNull('München') AS ascii;", - R"( -┌─ascii───────────────────────────┐ -│ xn--strae-oqa.xn--mnchen-3ya.de │ -└─────────────────────────────────┘ - )" - }} - }); - - factory.registerFunction>(FunctionDocumentation{ - .description=R"( -Computes a Unicode representation of an Internationalized Domain Name.)", - .syntax="idnaDecode(str)", - .arguments={{"str", "Input string"}}, - .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", - .examples={ - {"simple", - "SELECT idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de') AS unicode;", - R"( -┌─unicode───────────┐ -│ straße.münchen.de │ -└───────────────────┘ - )" - }} - }); } } diff --git a/tests/queries/0_stateless/02932_idna.reference b/tests/queries/0_stateless/02932_idna.reference index 8bfba53ad11b..0947194c07f2 100644 --- a/tests/queries/0_stateless/02932_idna.reference +++ b/tests/queries/0_stateless/02932_idna.reference @@ -1,24 +1,24 @@ -- Negative tests -- Regular cases -straße.de xn--strae-oqa.de straße.de xn--strae-oqa.de straße.de +straße.de xn--strae-oqa.de xn--strae-oqa.de straße.de straße.de 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 AMAZON amazon amazon amazon amazon aa-- aa-- aa-- aa-- aa-- -a†-- xn--a---kp0a a†-- xn--a---kp0a a†-- +a†-- xn--a---kp0a xn--a---kp0a a†-- a†-- ab--c ab--c ab--c ab--c ab--c --† xn----xhn -† xn----xhn -† --x.xn--zca -x.xn--zca -x.ß -x.xn--zca -x.ß -x-.xn--zca x-.xn--zca x-.ß x-.xn--zca x-.ß -x-.ß x-.xn--zca x-.ß x-.xn--zca x-.ß -x..ß x..xn--zca x..ß x..xn--zca x..ß +-† xn----xhn xn----xhn -† -† +-x.xn--zca -x.xn--zca -x.xn--zca -x.ß -x.ß +x-.xn--zca x-.xn--zca x-.xn--zca x-.ß x-.ß +x-.ß x-.xn--zca x-.xn--zca x-.ß x-.ß +x..ß x..xn--zca x..xn--zca x..ß x..ß 128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 -xn--zca.xn--zca xn--zca.xn--zca ß.ß xn--zca.xn--zca ß.ß -xn--zca.ß xn--zca.xn--zca ß.ß xn--zca.xn--zca ß.ß +xn--zca.xn--zca xn--zca.xn--zca xn--zca.xn--zca ß.ß ß.ß +xn--zca.ß xn--zca.xn--zca xn--zca.xn--zca ß.ß ß.ß x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x -x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß -x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß +x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.ß +x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.ß 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x -≠ xn--1ch ≠ xn--1ch ≠ +≠ xn--1ch xn--1ch ≠ ≠ aa-- aa-- aa-- aa-- ab--c ab--c ab--c ab--c -x -x -x -x @@ -54,22 +54,35 @@ xn--55qw42g.xn--55qw42g 公益.公益 xn--55qw42g.xn--55qw42g xn--55qw42g.xn--55 ≠ ≠ xn--1ch xn--1ch ファッション.biz ファッション.biz xn--bck1b9a5dre4c.biz xn--bck1b9a5dre4c.biz -- Special cases +---- Empty input +---- NULL input \N \N \N -\N -\N -\N -\N +---- Garbage inputs for idnaEncode + + + + +---- Long input +Row 1: +────── +idna: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +ascii: wenn sie ...xn-- vom hauptbahnhof in mnchen -n7c...xn-- mit zehn minuten, ohne, dass sie am flughafen noch einchecken mssen, dann starten sie im grunde genommen am flughafen -8gm... am ...xn-- am hauptbahnhof in mnchen starten sie ihren flug-0cf. zehn minuten.xn-- schauen sie sich mal die groen flughfen an, wenn sie in heathrow in london oder sonst wo, meine se -83h23c...xn-- charles de gaulle h in frankreich oder in -jvd...xn--h-zfa... in ... in...xn--h-zfa...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ...xn-- an den flughafen franz josef strau-z2c.xn-- dann starten sie praktisch hier am hauptbahnhof in mnchen-t9f.xn-- das bedeutet natrlich, dass der hauptbahnhof im grunde genommen nher an bayern -lxg23q...xn-- an die bayerischen stdte heranwchst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen-1hkk. +ascii_try: wenn sie ...xn-- vom hauptbahnhof in mnchen -n7c...xn-- mit zehn minuten, ohne, dass sie am flughafen noch einchecken mssen, dann starten sie im grunde genommen am flughafen -8gm... am ...xn-- am hauptbahnhof in mnchen starten sie ihren flug-0cf. zehn minuten.xn-- schauen sie sich mal die groen flughfen an, wenn sie in heathrow in london oder sonst wo, meine se -83h23c...xn-- charles de gaulle h in frankreich oder in -jvd...xn--h-zfa... in ... in...xn--h-zfa...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ...xn-- an den flughafen franz josef strau-z2c.xn-- dann starten sie praktisch hier am hauptbahnhof in mnchen-t9f.xn-- das bedeutet natrlich, dass der hauptbahnhof im grunde genommen nher an bayern -lxg23q...xn-- an die bayerischen stdte heranwchst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen-1hkk. +original: wenn sie ... vom hauptbahnhof in münchen ... mit zehn minuten, ohne, dass sie am flughafen noch einchecken müssen, dann starten sie im grunde genommen am flughafen ... am ... am hauptbahnhof in münchen starten sie ihren flug. zehn minuten. schauen sie sich mal die großen flughäfen an, wenn sie in heathrow in london oder sonst wo, meine se ... charles de gaulle äh in frankreich oder in ...äh... in ... in...äh...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ... an den flughafen franz josef strauß. dann starten sie praktisch hier am hauptbahnhof in münchen. das bedeutet natürlich, dass der hauptbahnhof im grunde genommen näher an bayern ... an die bayerischen städte heranwächst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen. +original_try: wenn sie ... vom hauptbahnhof in münchen ... mit zehn minuten, ohne, dass sie am flughafen noch einchecken müssen, dann starten sie im grunde genommen am flughafen ... am ... am hauptbahnhof in münchen starten sie ihren flug. zehn minuten. schauen sie sich mal die großen flughäfen an, wenn sie in heathrow in london oder sonst wo, meine se ... charles de gaulle äh in frankreich oder in ...äh... in ... in...äh...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ... an den flughafen franz josef strauß. dann starten sie praktisch hier am hauptbahnhof in münchen. das bedeutet natürlich, dass der hauptbahnhof im grunde genommen näher an bayern ... an die bayerischen städte heranwächst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen. +---- Non-const input -münchen xn--mnchen-3ya münchen xn--mnchen-3ya münchen -straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de -london.co.uk london.co.uk -microsoft.com microsoft.com -straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de -xn-- \N -xn-- \N -xn--tešla \N +münchen xn--mnchen-3ya xn--mnchen-3ya münchen münchen +straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de straße.münchen.de +---- Non-const input with invalid values sprinkled in +london.co.uk london.co.uk london.co.uk +microsoft.com microsoft.com microsoft.com +xn-- +xn-- +xn--tešla +ytraße.münchen.de xn--ytrae-oqa.xn--mnchen-3ya.de ytraße.münchen.de diff --git a/tests/queries/0_stateless/02932_idna.sql b/tests/queries/0_stateless/02932_idna.sql index 3572d4a6aeca..db7688064f28 100644 --- a/tests/queries/0_stateless/02932_idna.sql +++ b/tests/queries/0_stateless/02932_idna.sql @@ -6,118 +6,119 @@ SELECT '-- Negative tests'; SELECT idnaEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT idnaEncodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryIdnaEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT idnaDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT idnaEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT idnaEncodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tryIdnaEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT idnaDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT idnaEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT idnaEncodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryIdnaEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT idnaDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT idnaEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT idnaEncodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT idnaDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT tryIdnaEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT idnaDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } SELECT '-- Regular cases'; -- The test cases originate from the ada idna unit tests: -- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_alternating.txt -- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_unicode_alternating.txt - -SELECT 'straße.de' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '2001:4860:4860::8888' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'AMAZON' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'aa--' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'a†--' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'ab--c' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '-†' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '-x.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x-.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x-.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x..ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '128.0,0.1' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'xn--zca.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'xn--zca.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x01234567890123456789012345678901234567890123456789012345678901x' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.xn--zca' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.ß' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; -SELECT '≠' AS idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull; - -SELECT 'aa--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'ab--c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT '-x' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT '' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--1ch' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--dqd20apc' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--gdh' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--80aaa0ahbbeh4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--3bs854c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--mgb9awbf' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--mgbaam7a8h' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--mgbbh1a71e' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--s7y.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--55qx5d.xn--tckwe' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--4dbrk0ce' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--zckzah' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--p1ai.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--mxahbxey0c.gr' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--d1acpjx3f.xn--p1ai' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--sterreich-z7a.at' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--h2breg3eve.xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'ejemplo.xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--9t4b11yi5a.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--gk3at1e.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--42c2d9a' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT '1xn--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--bih.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--mgbb9fbpob' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'xn--55qw42g.xn--55qw42g' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT '≠' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; -SELECT 'ファッション.biz' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, idnaEncodeOrNull(unicode) AS originalOrNull; - +-- +SELECT 'straße.de' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '2001:4860:4860::8888' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'AMAZON' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'aa--' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'a†--' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'ab--c' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '-†' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '-x.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x-.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x-.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x..ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '128.0,0.1' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'xn--zca.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'xn--zca.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '≠' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; + +SELECT 'aa--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ab--c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '-x' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--1ch' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--dqd20apc' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--gdh' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--80aaa0ahbbeh4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--3bs854c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgb9awbf' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbaam7a8h' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbbh1a71e' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--s7y.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--55qx5d.xn--tckwe' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--4dbrk0ce' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--zckzah' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--p1ai.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mxahbxey0c.gr' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--d1acpjx3f.xn--p1ai' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--sterreich-z7a.at' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--h2breg3eve.xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ejemplo.xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--9t4b11yi5a.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--gk3at1e.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--42c2d9a' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '1xn--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--bih.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbb9fbpob' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--55qw42g.xn--55qw42g' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '≠' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ファッション.biz' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +-- SELECT '-- Special cases'; +SELECT '---- Empty input'; SELECT idnaEncode(''); -SELECT idnaEncodeOrNull(''); +SELECT tryIdnaEncode(''); SELECT idnaDecode(''); +SELECT '---- NULL input'; SELECT idnaEncode(NULL); -SELECT idnaEncodeOrNull(NULL); +SELECT tryIdnaEncode(NULL); SELECT idnaDecode(NULL); --- garbage IDNA/unicode values, see +SELECT '---- Garbage inputs for idnaEncode'; -- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_invalid.txt --- only idnaEncode() is tested, idnaDecode() has by definition no invalid input values SELECT idnaEncode('xn--'); -- { serverError BAD_ARGUMENTS } -SELECT idnaEncodeOrNull('xn--'); +SELECT tryIdnaEncode('xn--'); SELECT idnaEncode('ﻱa'); -- { serverError BAD_ARGUMENTS } -SELECT idnaEncodeOrNull('ﻱa'); +SELECT tryIdnaEncode('ﻱa'); SELECT idnaEncode('xn--a-yoc'); -- { serverError BAD_ARGUMENTS } -SELECT idnaEncodeOrNull('xn--a-yoc'); +SELECT tryIdnaEncode('xn--a-yoc'); SELECT idnaEncode('xn--tešla'); -- { serverError BAD_ARGUMENTS } -SELECT idnaEncodeOrNull('xn--tešla'); +SELECT tryIdnaEncode('xn--tešla'); --- long input --- SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS idna, idnaEncode(idna) AS ascii, idnaEncodeOrNull(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull FORMAT Vertical; +SELECT '---- Long input'; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(ascii) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try FORMAT Vertical; --- non-const values +SELECT '---- Non-const input'; DROP TABLE IF EXISTS tab; CREATE TABLE tab (idna String) ENGINE=MergeTree ORDER BY idna; INSERT INTO tab VALUES ('straße.münchen.de') ('') ('münchen'); -SELECT idna, idnaEncode(idna) AS ascii, idnaDecode(ascii) AS original, idnaEncodeOrNull(idna) AS asciiOrNull, idnaDecode(asciiOrNull) AS originalOrNull FROM tab; +SELECT idna, idnaEncode(idna) AS ascii, tryIdnaEncode(ascii) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try FROM tab; DROP TABLE tab; --- non-const values with a few invalid values for testing the OrNull variants +SELECT '---- Non-const input with invalid values sprinkled in'; DROP TABLE IF EXISTS tab; -CREATE TABLE tab (ascii String) ENGINE=MergeTree ORDER BY ascii; -INSERT INTO tab VALUES ('xn--') ('london.co.uk') ('straße.münchen.de') ('xn--tešla') ('microsoft.com') ('xn--'); -SELECT ascii, idnaEncode(ascii) AS original FROM tab; -- { serverError BAD_ARGUMENTS } -SELECT ascii, idnaEncodeOrNull(ascii) AS original FROM tab; +CREATE TABLE tab (idna String) ENGINE=MergeTree ORDER BY idna; +INSERT INTO tab VALUES ('xn--') ('london.co.uk') ('ytraße.münchen.de') ('xn--tešla') ('microsoft.com') ('xn--'); +SELECT idna, idnaEncode(idna) AS ascii FROM tab; -- { serverError BAD_ARGUMENTS } +SELECT idna, tryIdnaEncode(idna) AS ascii, idnaDecode(ascii) AS original FROM tab; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02932_punycode.reference b/tests/queries/0_stateless/02932_punycode.reference index 76508525b199..ff05eaa72a3b 100644 --- a/tests/queries/0_stateless/02932_punycode.reference +++ b/tests/queries/0_stateless/02932_punycode.reference @@ -1,52 +1,55 @@ -- Negative tests -- Regular cases -a a- a a- a -A A- A A- A --- --- -- --- -- -London London- London London- London -Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson -This has spaces This has spaces- This has spaces This has spaces- This has spaces --> $1.00 <- -> $1.00 <-- -> $1.00 <- -> $1.00 <-- -> $1.00 <- -а 80a а 80a а -ü tda ü tda ü -α mxa α mxa α -例 fsq 例 fsq 例 -😉 n28h 😉 n28h 😉 -αβγ mxacd αβγ mxacd αβγ -München Mnchen-3ya München Mnchen-3ya München -Mnchen-3ya Mnchen-3ya- Mnchen-3ya Mnchen-3ya- Mnchen-3ya -München-Ost Mnchen-Ost-9db München-Ost Mnchen-Ost-9db München-Ost -Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost -abæcdöef abcdef-qua4k abæcdöef abcdef-qua4k abæcdöef -правда 80aafi6cg правда 80aafi6cg правда -ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข -ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 -MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 -「bücher」 bcher-kva8445foa 「bücher」 bcher-kva8445foa 「bücher」 -团淄 3bs854c 团淄 3bs854c 团淄 +a a- a a +A A- A A +-- --- -- -- +London London- London London +Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson Lloyd-Atkinson +This has spaces This has spaces- This has spaces This has spaces +-> $1.00 <- -> $1.00 <-- -> $1.00 <- -> $1.00 <- +а 80a а а +ü tda ü ü +α mxa α α +例 fsq 例 例 +😉 n28h 😉 😉 +αβγ mxacd αβγ αβγ +München Mnchen-3ya München München +Mnchen-3ya Mnchen-3ya- Mnchen-3ya Mnchen-3ya +München-Ost Mnchen-Ost-9db München-Ost München-Ost +Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost Bahnhof München-Ost +abæcdöef abcdef-qua4k abæcdöef abæcdöef +правда 80aafi6cg правда правда +ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข ยจฆฟคฏข +ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 ドメイン名例 +MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 MajiでKoiする5秒前 +「bücher」 bcher-kva8445foa 「bücher」 「bücher」 +团淄 3bs854c 团淄 团淄 -- Special cases +---- Empty input - -\N -\N +---- NULL input \N \N \N +---- Garbage Punycode-encoded input + +---- Long input Row 1: ────── -str: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. -puny: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa -original: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. -punyOrNull: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa -originalOrNull: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. -München Mnchen-3ya München Mnchen-3ya München -abc abc- abc abc- abc -aäoöuü aou-qla5gqb aäoöuü aou-qla5gqb aäoöuü -Also no punycode \N +str: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +puny: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa +original: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +original_try: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +---- Non-const values +München Mnchen-3ya München München +abc abc- abc abc +aäoöuü aou-qla5gqb aäoöuü aäoöuü +---- Non-const values with invalid values sprinkled in +Also no punycode London- London Mnchen-3ya München -No punycode \N +No punycode Rtting-3ya Rütting -XYZ no punycode \N +XYZ no punycode diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql index 8df47cbf3da0..b9bcf933641b 100644 --- a/tests/queries/0_stateless/02932_punycode.sql +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -6,85 +6,81 @@ SELECT '-- Negative tests'; SELECT punycodeEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeEncodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecodeOrNull(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryPunycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeEncodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecodeOrNull(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tryPunycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeEncodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecodeOrNull('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryPunycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeEncodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecodeOrNull(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT tryPunycodeDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } SELECT '-- Regular cases'; -- The test cases originate from the ada idna unit tests: -- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt -SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; -SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull; - +SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +-- SELECT '-- Special cases'; +SELECT '---- Empty input'; SELECT punycodeEncode(''); -SELECT punycodeEncodeOrNull(''); SELECT punycodeDecode(''); -SELECT punycodeDecodeOrNull(''); +SELECT tryPunycodeDecode(''); +SELECT '---- NULL input'; SELECT punycodeEncode(NULL); -SELECT punycodeEncodeOrNull(NULL); SELECT punycodeDecode(NULL); -SELECT punycodeDecodeOrNull(NULL); +SELECT tryPunycodeDecode(NULL); --- garbage Punycode-encoded values +SELECT '---- Garbage Punycode-encoded input'; SELECT punycodeDecode('no punycode'); -- { serverError BAD_ARGUMENTS } -SELECT punycodeDecodeOrNull('no punycode'); +SELECT tryPunycodeDecode('no punycode'); --- long input -SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) as punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull FORMAT Vertical; +SELECT '---- Long input'; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try FORMAT Vertical; --- non-const values +SELECT '---- Non-const values'; DROP TABLE IF EXISTS tab; CREATE TABLE tab (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO tab VALUES ('abc') ('aäoöuü') ('München'); -SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, punycodeEncodeOrNull(str) AS punyOrNull, punycodeDecodeOrNull(punyOrNull) AS originalOrNull FROM tab; +SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try FROM tab; DROP TABLE tab; --- non-const values with a few invalid values for testing the OrNull variants +SELECT '---- Non-const values with invalid values sprinkled in'; DROP TABLE IF EXISTS tab; CREATE TABLE tab (puny String) ENGINE=MergeTree ORDER BY puny; INSERT INTO tab VALUES ('Also no punycode') ('London-') ('Mnchen-3ya') ('No punycode') ('Rtting-3ya') ('XYZ no punycode'); SELECT puny, punycodeDecode(puny) AS original FROM tab; -- { serverError BAD_ARGUMENTS } -SELECT puny, punycodeDecodeOrNull(puny) AS original FROM tab; +SELECT puny, tryPunycodeDecode(puny) AS original FROM tab; DROP TABLE tab; From 53f36f8b2104073ad19ccee9bdb2613d7cef16a2 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 7 Nov 2023 13:35:46 +0100 Subject: [PATCH 054/114] 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 000000000000..5fa644c20f94 --- /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 000000000000..1078c25774ce --- /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 ca5617c8aa20..feda1d9ac299 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 000000000000..7d81726344af --- /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 000000000000..29f2a4a22c25 --- /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 000000000000..74c17a87a10f --- /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 3271a1901938..881f7a46c168 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 beb8a4006327..6a0914718886 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 c8b3aeaca280..caa4beaba3b0 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 @@ class S3ObjectStorage : public IObjectStorage 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 @@ class S3ObjectStorage : public IObjectStorage private: std::string bucket; - String object_key_prefix; + ObjectStorageKeysGeneratorPtr key_generator; std::string disk_name; MultiVersion client; @@ -199,11 +199,6 @@ class S3ObjectStorage : public IObjectStorage 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 7543fb94331b..a35a1eb2a82f 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -91,6 +91,60 @@ class CheckAccess } }; +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 000000000000..408fa36fdd3e --- /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 31c6a3bf9686..cade16dc9c6d 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 000000000000..dc1bab458f00 --- /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 485bf73dad10..5f91c0e72efa 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 055/114] speed up generation --- src/Common/MatchGenerator.cpp | 192 +++++++++++------- .../tests/gtest_generate_random_by_regexp.cpp | 16 +- 2 files changed, 126 insertions(+), 82 deletions(-) diff --git a/src/Common/MatchGenerator.cpp b/src/Common/MatchGenerator.cpp index 5fa644c20f94..21762c65cff7 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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_)) - {} + RegexpConcatFunction(Children children_, const Generators & generators_) + : children(getFuncs(children_, generators_)) + { + } - String operator () (const Generators& generators_) override + String operator () () 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(); + } + String result; - for (auto child: children) + result.reserve(total_size); + for (auto & part: part_result) { - auto res = generators_.at(child)->operator()(generators_); - result.append(res); + 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 74c17a87a10f..70e806b023c9 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 056/114] 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 21762c65cff7..12afa0fd6e79 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -57,7 +57,8 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker { } - 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 @@ class RandomStringPrepareWalker : public Regexp::Walker { } - 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 @@ class RandomStringPrepareWalker : public Regexp::Walker { } - 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 @@ class RandomStringPrepareWalker : public Regexp::Walker } } - 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker } } - String operator () () override + size_t operator () (char * out, size_t size) override + { + chassert(literal_string.size() <= size); + + memcpy(out, literal_string.data(), literal_string.size()); + return literal_string.size(); + } + + size_t getRequiredSize() override { - return literal_string; + return literal_string.size(); } private: @@ -250,9 +288,17 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker { } - String operator () () override + size_t operator () (char *, size_t) override { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, @@ -275,6 +321,11 @@ class RandomStringPrepareWalker : public Regexp::Walker operation); } + size_t getRequiredSize() override + { + return 0; + } + private: String operation; }; @@ -296,14 +347,21 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 70e806b023c9..67ccf1ffe84e 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 057/114] 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 7d81726344af..ffac185ae3f1 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 @@ class GeneratorWithPrefix : public DB::IObjectStorageKeysGenerator 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 058/114] 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 | 209 +++++++++++++----- 9 files changed, 221 insertions(+), 81 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 6f0dabb52071..e372d4db465b 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 67056cc1bc13..beacd181e1d4 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 c3baf3fdbda6..6962248c7e12 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 000000000000..834f5102da16 --- /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 000000000000..2bb8d49ec4b0 --- /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 2f9fd44c9b05..8cbed155a723 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 cade16dc9c6d..e901f0df51d2 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 dc1bab458f00..c3b515e87777 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 5f91c0e72efa..ae87b19c0714 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}' - """ - - node_old.query(create_table_statement) - node_new.query(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 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() - - assert count_old == "2" - assert count_new == "2" - - node_old.query("DROP TABLE test_replicated_merge_tree SYNC") - node_new.query("DROP TABLE test_replicated_merge_tree SYNC") + 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}' + """ + + 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 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() + + 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() + + 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 059/114] 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 ae87b19c0714..8c52b05dba20 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 count_lines_with(lines, pattern): 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 060/114] 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 c9ce56971827..9079246429f5 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 e372d4db465b..6f0dabb52071 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 beacd181e1d4..bca8800ab2bf 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 12afa0fd6e79..bc96634b5180 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -188,7 +188,7 @@ class RandomStringPrepareWalker : public Regexp::Walker using CharRanges = std::vector>; public: - RegexpCharClassFunction(Regexp * re_) + explicit RegexpCharClassFunction(Regexp * re_) { CharClass * cc = re_->cc(); chassert(cc); @@ -198,7 +198,7 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker class RegexpLiteralStringFunction : public NodeFunction { public: - RegexpLiteralStringFunction(Regexp * re_) + explicit RegexpLiteralStringFunction(Regexp * re_) { if (re_->nrunes() == 0) return; @@ -279,7 +279,7 @@ class RandomStringPrepareWalker : public Regexp::Walker class RegexpLiteralFunction : public NodeFunction { public: - RegexpLiteralFunction(Regexp * re_) + explicit RegexpLiteralFunction(Regexp * re_) { char buffer[UTFmax]; @@ -308,7 +308,7 @@ class RandomStringPrepareWalker : public Regexp::Walker class ThrowExceptionFunction : public NodeFunction { public: - ThrowExceptionFunction(Regexp * re_) + explicit ThrowExceptionFunction(Regexp * re_) : operation(magic_enum::enum_name(re_->op())) { } @@ -332,7 +332,7 @@ class RandomStringPrepareWalker : public Regexp::Walker public: - RandomStringPrepareWalker(bool logging) + explicit RandomStringPrepareWalker(bool logging) : logger(logging ? &Poco::Logger::get("GeneratorCombiner") : nullptr) { if (logger) @@ -344,7 +344,7 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker return pre_arg; } -private: Poco::Logger * logger = nullptr; Regexp * root = nullptr; diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index ffac185ae3f1..deae9c67e5eb 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -15,7 +15,6 @@ class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator , re_gen(key_template, /*logging*/ false) { } - DB::ObjectStorageKey generate(const String &) const override { return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); @@ -30,7 +29,7 @@ class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator 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 @@ class GeneratorWithPrefix : public DB::IObjectStorageKeysGenerator /// 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 4d81161b6de6..89fcb9ce3506 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 6de0614541ad..49699b7d2fd7 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 8cbed155a723..2fd72245ecfb 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 061/114] 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 bc96634b5180..a2f7529cbb39 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 1078c25774ce..65bfe0c92089 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 deae9c67e5eb..c970b193adbd 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 062/114] 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 ed413959ca6c..d4251e7e74c8 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 063/114] 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 2fd72245ecfb..73562f58b505 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 064/114] 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 73562f58b505..3a0744a298d5 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 065/114] 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 a2f7529cbb39..10044b59a35c 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -364,7 +364,7 @@ class RandomStringPrepareWalker : public Regexp::Walker } 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 066/114] 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 10044b59a35c..e736cd5195ca 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -377,7 +377,7 @@ class RandomStringPrepareWalker : public Regexp::Walker { 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 067/114] 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 e736cd5195ca..b6562c3aacf4 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -228,8 +228,7 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 068/114] 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 b6562c3aacf4..9b525bb73da8 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -394,7 +394,7 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 069/114] 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 9b525bb73da8..5495d58d65f0 100644 --- a/src/Common/MatchGenerator.cpp +++ b/src/Common/MatchGenerator.cpp @@ -379,7 +379,7 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 22474f6b38c490db750148dc8c72839f31f46e31 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Fri, 5 Jan 2024 11:39:52 -0800 Subject: [PATCH 070/114] changed 2nd argument type to UInt and other cosmetic changes --- .../functions/time-series-functions.md | 10 +-- src/Common/config.h.in | 1 - src/Functions/seriesDecomposeSTL.cpp | 79 +++++++------------ src/Functions/stl.hpp | 3 + ...nce => 02813_seriesDecomposeSTL.reference} | 0 ...mpose.sql => 02813_seriesDecomposeSTL.sql} | 10 +-- 6 files changed, 41 insertions(+), 62 deletions(-) rename tests/queries/0_stateless/{02813_series_decompose.reference => 02813_seriesDecomposeSTL.reference} (100%) rename tests/queries/0_stateless/{02813_series_decompose.sql => 02813_seriesDecomposeSTL.sql} (85%) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 84d5acb098a7..59c261df8068 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -4,7 +4,7 @@ sidebar_position: 172 sidebar_label: Time Series --- -#Time Series Functions +# Time Series Functions Below functions are used for time series analysis. @@ -60,10 +60,7 @@ Result: ## seriesDecomposeSTL -Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) -Returns an array of three arrays where the first array include seasonal components, the second array - trend, -and the third array - residue component. -https://www.wessa.net/download/stl.pdf +Decomposes a time series using STL [(Seasonal-Trend Decomposition Procedure Based on Loess)](https://www.wessa.net/download/stl.pdf) into a season, a trend and a residual component. **Syntax** @@ -80,7 +77,8 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- Array of arrays +- An array of three arrays where the first array include seasonal components, the second array - trend, +and the third array - residue component. Type: [Array](../../sql-reference/data-types/array.md). diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 8f04f7a1eb8f..f84e28942c54 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,7 +59,6 @@ #cmakedefine01 USE_PRQL #cmakedefine01 USE_OPENSSL_INTREE #cmakedefine01 USE_ULID -#cmakedefine01 USE_SEASONAL #cmakedefine01 FIU_ENABLE #cmakedefine01 USE_BCRYPT #cmakedefine01 USE_LIBARCHIVE diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index b6303f8cda6b..2739c7b71779 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -1,5 +1,3 @@ -#include "config.h" - #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wold-style-cast" @@ -54,8 +52,8 @@ class FunctionSeriesDecomposeSTL : public IFunction DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"time-series", &isArray, nullptr, "Array"}, - {"period", &isNativeNumber, nullptr, "Number"}, + {"time_series", &isArray, nullptr, "Array"}, + {"period", &isNativeUInt, nullptr, "Unsigned Integer"}, }; validateFunctionArgumentTypes(*this, arguments, args); @@ -68,8 +66,7 @@ class FunctionSeriesDecomposeSTL : public IFunction const ColumnArray * array = checkAndGetColumn(array_ptr.get()); if (!array) { - const ColumnConst * const_array = checkAndGetColumnConst( - arguments[0].column.get()); + const ColumnConst * const_array = checkAndGetColumnConst(arguments[0].column.get()); if (!const_array) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); @@ -81,13 +78,13 @@ class FunctionSeriesDecomposeSTL : public IFunction const IColumn & src_data = array->getData(); const ColumnArray::Offsets & src_offsets = array->getOffsets(); - Float64 period; + UInt64 period; - auto ret = ColumnFloat32::create(); - auto & res_data = ret->getData(); + auto res = ColumnFloat32::create(); + auto & res_data = res->getData(); - ColumnArray::ColumnOffsets::MutablePtr col_offsets = ColumnArray::ColumnOffsets::create(); - auto & col_offsets_data = col_offsets->getData(); + ColumnArray::ColumnOffsets::MutablePtr res_col_offsets = ColumnArray::ColumnOffsets::create(); + auto & res_col_offsets_data = res_col_offsets->getData(); auto root_offsets = ColumnArray::ColumnOffsets::create(); auto & root_offsets_data = root_offsets->getData(); @@ -97,19 +94,11 @@ class FunctionSeriesDecomposeSTL : public IFunction for (size_t i = 0; i < src_offsets.size(); ++i) { auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); - if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get()) - || checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) + if (checkAndGetColumn(period_ptr.get()) + || checkAndGetColumn(period_ptr.get()) + || checkAndGetColumn(period_ptr.get()) + || checkAndGetColumn(period_ptr.get())) period = period_ptr->getUInt(i); - else if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get())) - { - period = period_ptr->getFloat64(i); - if (isNaN(period) || !std::isfinite(period) || period < 0) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal value {} for second argument of function {}. Should be a positive number", - period, - getName()); - } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -136,15 +125,15 @@ class FunctionSeriesDecomposeSTL : public IFunction || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)) { res_data.insert(res_data.end(), seasonal.begin(), seasonal.end()); - col_offsets_data.push_back(res_data.size()); + res_col_offsets_data.push_back(res_data.size()); res_data.insert(res_data.end(), trend.begin(), trend.end()); - col_offsets_data.push_back(res_data.size()); + res_col_offsets_data.push_back(res_data.size()); res_data.insert(res_data.end(), residue.begin(), residue.end()); - col_offsets_data.push_back(res_data.size()); + res_col_offsets_data.push_back(res_data.size()); - root_offsets_data.push_back(col_offsets->size()); + root_offsets_data.push_back(res_col_offsets->size()); prev_src_offset = curr_offset; } @@ -155,14 +144,14 @@ class FunctionSeriesDecomposeSTL : public IFunction arguments[0].column->getName(), getName()); } - ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(ret), std::move(col_offsets)); + ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(res), std::move(res_col_offsets)); return ColumnArray::create(std::move(nested_array_col), std::move(root_offsets)); } template bool executeNumber( const IColumn & src_data, - Float64 period, + UInt64 period, ColumnArray::Offset & start, ColumnArray::Offset & end, std::vector & seasonal, @@ -179,38 +168,28 @@ class FunctionSeriesDecomposeSTL : public IFunction size_t len = end - start; if (len < 4) throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName()); - else if (period > (len / 2.0)) + if (period > (len / 2)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The series should have data of at least two period lengths for function {}", getName()); - std::vector src((src_vec.begin() + start), (src_vec.begin() + end)); - - try - { - auto res = stl::params().fit(src, static_cast(std::round(period))); + std::vector src(src_vec.begin() + start, src_vec.begin() + end); - if (res.seasonal.empty()) - return false; + auto res = stl::params().fit(src, static_cast(period)); - seasonal = res.seasonal; - trend = res.trend; - residue = res.remainder; - return true; - } - catch (...) - { + if (res.seasonal.empty()) return false; - } + + seasonal = std::move(res.seasonal); + trend = std::move(res.trend); + residue = std::move(res.remainder); + return true; } }; REGISTER_FUNCTION(seriesDecomposeSTL) { factory.registerFunction(FunctionDocumentation{ .description = R"( -Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) -Returns an array of three arrays where the first array include seasonal components, the second array - trend, -and the third array - residue component. -https://www.wessa.net/download/stl.pdf +Decomposes a time series using STL [(Seasonal-Trend Decomposition Procedure Based on Loess)](https://www.wessa.net/download/stl.pdf) into a season, a trend and a residual component. **Syntax** @@ -227,7 +206,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- Array of arrays +- An array of three arrays where the first array include seasonal components, the second array - trend, and the third array - residue component. Type: [Array](../../sql-reference/data-types/array.md). diff --git a/src/Functions/stl.hpp b/src/Functions/stl.hpp index 5b257b1c59de..fbfc52f959ca 100644 --- a/src/Functions/stl.hpp +++ b/src/Functions/stl.hpp @@ -1,3 +1,6 @@ +// Dump of https://github.com/ankane/stl-cpp/blob/3b1b3a3e9335cda26c8b0797d8b8d24ac8e350ad/include/stl.hpp. +// Added to ClickHouse source code and not referenced as a submodule because its easier maintain and modify/customize. + /*! * STL C++ v0.1.3 * https://github.com/ankane/stl-cpp diff --git a/tests/queries/0_stateless/02813_series_decompose.reference b/tests/queries/0_stateless/02813_seriesDecomposeSTL.reference similarity index 100% rename from tests/queries/0_stateless/02813_series_decompose.reference rename to tests/queries/0_stateless/02813_seriesDecomposeSTL.reference diff --git a/tests/queries/0_stateless/02813_series_decompose.sql b/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql similarity index 85% rename from tests/queries/0_stateless/02813_series_decompose.sql rename to tests/queries/0_stateless/02813_seriesDecomposeSTL.sql index abaa99b38171..929d0474e094 100644 --- a/tests/queries/0_stateless/02813_series_decompose.sql +++ b/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql @@ -1,18 +1,18 @@ --- Tags: no-fasttest, no-cpu-aarch64 +-- Tags: no-cpu-aarch64 -- Tag no-cpu-aarch64: values generated are slighly different on aarch64 DROP TABLE IF EXISTS tb2; -CREATE TABLE tb2 (`n` UInt32, `a` Array(Float64)) ENGINE = Memory; +CREATE TABLE tb2 (`period` UInt32, `ts` Array(Float64)) ENGINE = Memory; INSERT INTO tb2 VALUES (3,[10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34]); INSERT INTO tb2 VALUES (14, [139, 87, 110, 68, 54, 50, 51, 53, 133, 86, 141, 97, 156, 94, 149, 95, 140, 77, 61, 50, 54, 47, 133, 72, 152, 94, 148, 105, 162, 101, 160, 87, 63, 53, 55, 54, 151, 103, 189, 108, 183, 113, 175, 113, 178, 90, 71, 62, 62, 65, 165, 109, 181, 115, 182, 121, 178, 114, 170]); SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3); SELECT seriesDecomposeSTL([2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], 0); -SELECT seriesDecomposeSTL(a, n) FROM tb2; +SELECT seriesDecomposeSTL(ts, period) FROM tb2 ORDER BY period; DROP TABLE IF EXISTS tb2; -SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5); -- { serverError ILLEGAL_COLUMN} -SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5.2); --{ serverError ILLEGAL_COLUMN} +SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT seriesDecomposeSTL([2,2,2,2,2,2,2,2,2,2,2,2,2,2], -5.2); --{ serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT seriesDecomposeSTL(); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT seriesDecomposeSTL([]); --{ serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT seriesDecomposeSTL([1,2,3], 2); --{ serverError BAD_ARGUMENTS} From 24fe5f4d534ed8ec2b3df36514660af08faa5367 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 5 Jan 2024 20:41:58 +0000 Subject: [PATCH 071/114] 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 746f419fa1d1..ed1e65a6b648 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.storage_limits = std::make_shared(storage_limits); + 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 072/114] 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 f3917b878d6b..5c068ae26dfb 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 073/114] 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 | 210 ++++++++++-------- 12 files changed, 165 insertions(+), 178 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 8b7f86cce5c5..8ebab80423fb 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 2613e9ec1166..724b6ab62f7c 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 119e0d991431..101959dbf83d 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 910615bf6efb..716184e07d0c 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 a65a82d9b408..b65de8d34a73 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 b137f0605dc2..677b739fd396 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -142,18 +142,7 @@ class Client : private Aws::S3::S3Client 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 @@ class Client : private Aws::S3::S3Client 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 96ad6413ef51..f2ead22127c5 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 ebfc07a39767..6ee8d96ed090 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 60ae7f219f43..0909539033d8 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 7dac6d9fbb57..000000000000 --- 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 64674e2a3e3f..000000000000 --- 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 b83c095a7a6b..6fee339c29be 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.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__)) - - -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/*", - }, - ], - } +DEFAULT_AUTH = ["'minio'", "'minio123'"] +NO_AUTH = ["NOSIGN"] + + +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 dd2d9ff168d3aad612dcbcec71a003faf9750b03 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 08:25:19 +0000 Subject: [PATCH 074/114] Minor fixes --- src/Functions/idna.cpp | 18 +++++++++--------- src/Functions/punycode.cpp | 4 ++-- .../aspell-ignore/en/aspell-dict.txt | 5 ++--- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Functions/idna.cpp b/src/Functions/idna.cpp index 087131021c9a..a73347400c64 100644 --- a/src/Functions/idna.cpp +++ b/src/Functions/idna.cpp @@ -30,7 +30,6 @@ namespace ErrorCodes /// - idnaEncode(), tryIdnaEncode() and idnaDecode(), see https://en.wikipedia.org/wiki/Internationalized_domain_name#ToASCII_and_ToUnicode /// and [3] https://www.unicode.org/reports/tr46/#ToUnicode - enum class ErrorHandling { Throw, /// Throw exception @@ -71,7 +70,7 @@ struct IdnaEncode { if constexpr (error_handling == ErrorHandling::Throw) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to Punycode", std::string_view(value, value_length)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to ASCII", value_view); } else { @@ -96,6 +95,7 @@ struct IdnaEncode } }; +/// Translates an ASII-encoded IDNA string back to its UTF-8 representation. struct IdnaDecode { /// As per the specification, invalid inputs are returned as is, i.e. there is no special error handling. @@ -113,11 +113,11 @@ struct IdnaDecode std::string unicode; for (size_t row = 0; row < rows; ++row) { - const char * value = reinterpret_cast(&data[prev_offset]); - const size_t value_length = offsets[row] - prev_offset - 1; - std::string_view value_view(value, value_length); + const char * ascii = reinterpret_cast(&data[prev_offset]); + const size_t ascii_length = offsets[row] - prev_offset - 1; + std::string_view ascii_view(ascii, ascii_length); - unicode = ada::idna::to_unicode(value_view); + unicode = ada::idna::to_unicode(ascii_view); res_data.insert(unicode.c_str(), unicode.c_str() + unicode.size() + 1); res_offsets.push_back(res_data.size()); @@ -149,7 +149,7 @@ REGISTER_FUNCTION(Idna) Computes an ASCII representation of an Internationalized Domain Name. Throws an exception in case of error.)", .syntax="idnaEncode(str)", .arguments={{"str", "Input string"}}, - .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .returned_value="An ASCII-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", .examples={ {"simple", "SELECT idnaEncode('straße.münchen.de') AS ascii;", @@ -166,7 +166,7 @@ Computes an ASCII representation of an Internationalized Domain Name. Throws an Computes a ASCII representation of an Internationalized Domain Name. Returns an empty string in case of error)", .syntax="punycodeEncode(str)", .arguments={{"str", "Input string"}}, - .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .returned_value="An ASCII-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", .examples={ {"simple", "SELECT idnaEncodeOrNull('München') AS ascii;", @@ -180,7 +180,7 @@ Computes a ASCII representation of an Internationalized Domain Name. Returns an factory.registerFunction(FunctionDocumentation{ .description=R"( -Computes a Unicode representation of an Internationalized Domain Name.)", +Computes the Unicode representation of ASCII-encoded Internationalized Domain Name.)", .syntax="idnaDecode(str)", .arguments={{"str", "Input string"}}, .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index 66e5ac6cb7b9..159189744bdd 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -37,7 +37,7 @@ enum class ErrorHandling struct PunycodeEncode { - /// Encoding-as-punycode can only fail if the input isn't valid UTF8. In that case, returnn undefined output, i.e. garbage-in, garbage-out. + /// Encoding-as-punycode can only fail if the input isn't valid UTF8. In that case, return undefined output, i.e. garbage-in, garbage-out. static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -60,7 +60,7 @@ struct PunycodeEncode value_utf32.resize(value_utf32_length); const size_t codepoints = ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); if (codepoints == 0) - value_utf32.clear(); /// input was empty or it is not valid UTF-8 + value_utf32.clear(); /// input was empty or no valid UTF-8 const bool ok = ada::idna::utf32_to_punycode(value_utf32, value_puny); if (!ok) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 11fab98cf2df..dcff1c824447 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1657,7 +1657,6 @@ icudata idempotency idnaDecode idnaEncode -idnaEncodeOrNull ifNotFinite ifNull iframe @@ -2083,9 +2082,7 @@ pseudorandomize psql ptrs punycodeDecode -punycodeDecodeOrNull punycodeEncode -punycodeEncodeOrNull pushdown pwrite py @@ -2532,6 +2529,8 @@ trimRight trunc tryBase tryDecrypt +tryIdnaEncode +tryPunycodeDecode tskv tsv tui From e947ed710b10a0d2a5266f196a751ca9cc57b943 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 13:03:21 +0000 Subject: [PATCH 075/114] Hijack and document 'update' map to represent reload status --- .../system-tables/server_settings.md | 39 ++-- src/Core/ServerSettings.cpp | 4 +- src/Core/ServerSettings.h | 206 ++++++++---------- .../System/StorageSystemServerSettings.cpp | 94 ++++---- 4 files changed, 170 insertions(+), 173 deletions(-) diff --git a/docs/en/operations/system-tables/server_settings.md b/docs/en/operations/system-tables/server_settings.md index 6374b2d02a2b..417c3460a532 100644 --- a/docs/en/operations/system-tables/server_settings.md +++ b/docs/en/operations/system-tables/server_settings.md @@ -14,12 +14,11 @@ Columns: - `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml` - `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description. - `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type. -- `is_hot_reloadable` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether hot reload is supported -- `runtime_reload` ([Enum8](../../sql-reference/data-types/enum.md)) — Hot reload type of parameter. Possible values: - - `Yes` - - `OnlyIncrease` - - `OnlyDecrease` - - `No` +- `changeable_without_restart` ([Enum8](../../sql-reference/data-types/enum.md)) — Whether the setting can be changed at server runtime. Values: + - `'No' ` + - `'IncreaseOnly'` + - `'DecreaseOnly'` + - `'Yes'` - `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete. **Example** @@ -33,20 +32,20 @@ WHERE name LIKE '%thread_pool%' ``` ``` text -┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─is_hot_reloadable─┬─runtime_reload─┬─is_obsolete─┐ -│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │ No │ 0 │ -│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │ No │ 0 │ -│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │ No │ 0 │ -│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │ Yes │ 0 │ -│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ -│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ -│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │ Yes │ 0 │ -│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │ Yes │ 0 │ -│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │ Yes │ 0 │ -│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │ Yes │ 0 │ -│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ -│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │ Yes │ 0 │ -└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴───────────────────┴────────────────┴─────────────┘ +┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─changeable_without_restart─┬─is_obsolete─┐ +│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ No │ 0 │ +│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ No │ 0 │ +│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ No │ 0 │ +│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ No │ 0 │ +│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ No │ 0 │ +│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ No │ 0 │ +│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ No │ 0 │ +│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ No │ 0 │ +│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ No │ 0 │ +│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ No │ 0 │ +│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ No │ 0 │ +│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ No │ 0 │ +└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴────────────────────────────┴─────────────┘ ``` diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 7f48226c2139..fbf86d3e9adb 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -4,12 +4,10 @@ namespace DB { -IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WRAP) +IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config) { - - SERVER_SETTINGS(SET_RUNTIME_RELOAD, "", "") // settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h std::unordered_set settings_from_profile_allowlist = { "background_pool_size", diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 3215a35b4831..03a51c1f0659 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -9,132 +9,114 @@ namespace Poco::Util { class AbstractConfiguration; } -namespace DB -{ -enum class RuntimeReloadType +namespace DB { - Yes, - OnlyIncrease, - OnlyDecrease, - No, -}; - -#define SET_RUNTIME_RELOAD(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ - this->runtime_reload_map.insert(std::make_pair(#NAME, RUNTIME_RELOAD)); -#define M_WRAP(M, TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS, RUNTIME_RELOAD) \ - M(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - - -#define SERVER_SETTINGS(MW, M, ALIAS) \ - MW(M, Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0, RuntimeReloadType::No) \ - MW(M, Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0, RuntimeReloadType::No) \ - MW(M, Int32, max_connections, 1024, "Max server connections.", 0, RuntimeReloadType::No) \ - MW(M, UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0, RuntimeReloadType::No) \ - MW(M, UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0, RuntimeReloadType::No) \ - MW(M, String, default_database, "default", "Default database name.", 0, RuntimeReloadType::No) \ - MW(M, String, tmp_policy, "", "Policy for storage with temporary data.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0, RuntimeReloadType::No) \ - MW(M, String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ - MW(M, Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ - MW(M, Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0, RuntimeReloadType::Yes) \ - MW(M, Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0, RuntimeReloadType::No) \ - MW(M, Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0, RuntimeReloadType::No) \ +#define SERVER_SETTINGS(M, ALIAS) \ + M(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \ + M(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \ + M(UInt64, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0) \ + M(UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0) \ + M(UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0) \ + M(UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0) \ + M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \ + M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ + M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ + M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \ + M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \ + M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ + M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ + M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \ + M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \ + M(UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0) \ + M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \ + M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \ + M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \ + M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \ + M(UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0) \ + M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \ + M(Int32, max_connections, 1024, "Max server connections.", 0) \ + M(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ + M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0) \ + M(String, default_database, "default", "Default database name.", 0) \ + M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ + M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0) \ + M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \ + M(UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0) \ + M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0) \ + M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0) \ + M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \ + M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \ + M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ + M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ + M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ \ - MW(M, UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0, RuntimeReloadType::Yes) \ + M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ \ - MW(M, Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0, RuntimeReloadType::No) \ - MW(M, String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0, RuntimeReloadType::No) \ - MW(M, Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ - MW(M, String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0, RuntimeReloadType::No) \ - MW(M, Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ - MW(M, String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0, RuntimeReloadType::No) \ - MW(M, Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ - MW(M, String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0, RuntimeReloadType::No) \ - MW(M, Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0, RuntimeReloadType::No) \ + M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \ + M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ + M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ + M(Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0) \ + M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \ + M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ + M(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \ + M(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \ + M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \ + M(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \ + M(String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0) \ + M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0) \ + M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \ + M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ \ - MW(M, Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0, RuntimeReloadType::No) \ - MW(M, Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0, RuntimeReloadType::No) \ - MW(M, UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0, RuntimeReloadType::No) \ + M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ + M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ + M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \ \ - MW(M, UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0, RuntimeReloadType::OnlyDecrease) \ - MW(M, UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0, RuntimeReloadType::OnlyDecrease) \ + M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ + M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ + M(UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ + M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ - MW(M, UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0, RuntimeReloadType::OnlyIncrease) \ - MW(M, UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0, RuntimeReloadType::Yes) \ - MW(M, UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0, RuntimeReloadType::Yes) \ - MW(M, Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0, RuntimeReloadType::No) \ - MW(M, Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0, RuntimeReloadType::No) \ + M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \ + M(Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \ + M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \ + M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \ + M(UInt64, background_fetches_pool_size, 16, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \ + M(UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0) \ + M(UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0) \ + M(UInt64, background_schedule_pool_size, 512, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ + M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ + M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ + M(UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \ + M(UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \ + M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ \ - MW(M, Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0, RuntimeReloadType::No) \ - MW(M, Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ - MW(M, Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ - MW(M, Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0, RuntimeReloadType::No) \ - MW(M, Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::No) \ - MW(M, UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0, RuntimeReloadType::No) \ - MW(M, Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0, RuntimeReloadType::No) \ - MW(M, Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0, RuntimeReloadType::No) \ - - -#define SERVER_SETTINGS_WRAP(M, ALIAS) \ - SERVER_SETTINGS(M_WRAP, M, ALIAS) + M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ + M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ + M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ + M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ + M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ + M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \ + M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ -DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS_WRAP) + /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp +DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) struct ServerSettings : public BaseSettings { - std::unordered_map runtime_reload_map; void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config); }; diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index bcfca59618a3..7c42c610c221 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -1,12 +1,15 @@ -#include #include -#include +#include #include +#include #include +#include +#include #include #include -#include +#include #include +#include namespace CurrentMetrics @@ -20,18 +23,25 @@ namespace CurrentMetrics namespace DB { -static std::vector> getRuntimeReloadEnumAndValues() +enum class ChangeableWithoutRestart { - return std::vector>{ - {"Yes", static_cast(RuntimeReloadType::Yes)}, - {"OnlyIncrease", static_cast(RuntimeReloadType::OnlyIncrease)}, - {"OnlyDecrease", static_cast(RuntimeReloadType::OnlyDecrease)}, - {"No", static_cast(RuntimeReloadType::No)}, - }; -} + No, + IncreaseOnly, + DecreaseOnly, + Yes +}; NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() { + auto changeable_without_restart_type = std::make_shared( + DataTypeEnum8::Values + { + {"No", static_cast(ChangeableWithoutRestart::No)}, + {"IncreaseOnly", static_cast(ChangeableWithoutRestart::IncreaseOnly)}, + {"DecreaseOnly", static_cast(ChangeableWithoutRestart::DecreaseOnly)}, + {"Yes", static_cast(ChangeableWithoutRestart::Yes)}, + }); + return { {"name", std::make_shared()}, {"value", std::make_shared()}, @@ -39,38 +49,45 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, - {"is_hot_reloadable", std::make_shared()}, - {"runtime_reload", std::make_shared(getRuntimeReloadEnumAndValues())}, + {"changeable_without_restart", std::move(changeable_without_restart_type)}, {"is_obsolete", std::make_shared()} }; } void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - // Server settings that have been reloaded from the config file. - std::unordered_map updated = { - {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, - {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, - - {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, - {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, - - {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, - {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, - {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, - - {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, - {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, - {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, - {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} + /// When the server configuration file is periodically re-loaded from disk, the server components (e.g. memory tracking) are updated + /// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the + /// current setting values, one needs to ask the components directly. + std::unordered_map> changeable_settings = { + {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, + {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}}, + + {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, + {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, + + {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}}, + {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}}, + {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}}, + + {"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_message_broker_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_distributed_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + + {"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_uncompressed_cache_size", {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}} }; if (context->areBackgroundExecutorsInitialized()) { - updated.insert({"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}); - updated.insert({"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}); - updated.insert({"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}); - updated.insert({"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}); + changeable_settings.insert({"background_pool_size", {std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_move_pool_size", {std::to_string(context->getMovesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); } const auto & config = context->getConfigRef(); @@ -80,17 +97,18 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context for (const auto & setting : settings.all()) { const auto & setting_name = setting.getName(); - const auto & it = updated.find(setting_name); + + const auto & changeable_settings_it = changeable_settings.find(setting_name); + const bool is_changeable = (changeable_settings_it != changeable_settings.end()); res_columns[0]->insert(setting_name); - res_columns[1]->insert((it != updated.end()) ? it->second: setting.getValueString()); + res_columns[1]->insert(is_changeable ? changeable_settings_it->second.first : setting.getValueString()); res_columns[2]->insert(setting.getDefaultValueString()); res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); - res_columns[6]->insert((it != updated.end()) ? true : false); - res_columns[7]->insert(static_cast(settings.runtime_reload_map.contains(setting_name) ? settings.runtime_reload_map.at(setting_name): RuntimeReloadType::No)); - res_columns[8]->insert(setting.isObsolete()); + res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No); + res_columns[7]->insert(setting.isObsolete()); } } From 8cd85520861607cd8bdb07856cba81f38ea33ff9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jan 2024 14:13:49 +0000 Subject: [PATCH 076/114] Empty-Commit From 371d6646677adc522a67b8293c0dec0dd035df93 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 7 Jan 2024 17:27:22 +0100 Subject: [PATCH 077/114] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e6b0c581f277..4ad6c564a186 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4006,9 +4006,13 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW /// We don't need to commit it to zk, and don't even need to activate it. MergeTreePartInfo empty_info = drop_range; - empty_info.min_block = empty_info.level = empty_info.mutation = 0; + empty_info.level = empty_info.mutation = 0; + empty_info.min_block = MergeTreePartInfo::MAX_BLOCK_NUMBER; for (const auto & part : parts_to_remove) { + /// We still have to take min_block into account to avoid creating multiple covering ranges + /// that intersect each other + empty_info.min_block = std::min(empty_info.min_block, part->info.min_block); empty_info.level = std::max(empty_info.level, part->info.level); empty_info.mutation = std::max(empty_info.mutation, part->info.mutation); } From 6bc27dfcf58e770b92f002e0a53a346e93d40524 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 Jan 2024 20:56:16 +0100 Subject: [PATCH 078/114] Enable all access control improvements by default --- programs/server/config.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 1be20c5cad85..e1428b170844 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -713,11 +713,11 @@ For example, if there two users A, B and a row policy is defined only for A, then if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows. By default this setting is false for compatibility with earlier access configurations. --> - false + true - false + true - false + true - false + true - false + true 600 From 6f626d829433d9001fa9f5f900ef394c75eb930c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 7 Jan 2024 23:00:26 +0100 Subject: [PATCH 079/114] 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 f2ead22127c5..5039059f5229 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 0909539033d8..086cb0acbae4 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 2c61015578e892d5a839c11d426eb8fa8944ccc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 Jan 2024 23:06:36 +0100 Subject: [PATCH 080/114] Do not load database engines in suggest --- src/Client/Suggest.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 836c03d81ff9..eb98c3a57402 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -77,7 +77,6 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti }; add_column("name", "functions", false, {}); - add_column("name", "database_engines", false, {}); add_column("name", "table_engines", false, {}); add_column("name", "formats", false, {}); add_column("name", "table_functions", false, {}); From eb8217efb07dbf6306fd67b43028e9b89a52af99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 Jan 2024 01:22:11 +0100 Subject: [PATCH 081/114] Fix wrong message in Keeper --- programs/keeper/Keeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 48d26233d94a..109884ec899b 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -335,7 +335,7 @@ try else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) { throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly", + "By default 'keeper_server.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper_server.storage_path' in the keeper configuration explicitly", KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); } else From 29173a602e45391cf57bd152cd638ffe4719d2b7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Jan 2024 02:39:19 +0000 Subject: [PATCH 082/114] 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 6fee339c29be..7d40060fec6c 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -21,9 +21,7 @@ def create_bucket(client, bucket_name, policy): 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 wait_all_processed(files_num): 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 wait_all_processed(files_num): 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 149de9aa9378571eff9569cb90c9bcfd42709316 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 8 Jan 2024 03:40:48 +0000 Subject: [PATCH 083/114] discard fs::canonical return Signed-off-by: Duc Canh Le --- src/Storages/StorageFile.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b3c686c290dc..f44f25695f82 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -115,10 +115,10 @@ void listFilesWithRegexpMatchingImpl( { try { - /// We use fs::canonical to check if the file exists but the result path - /// will be fs::absolute. + /// We use fs::canonical to resolve the canonical path and check if the file does exists + /// but the result path will be fs::absolute. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. - fs::path canonical_path = fs::canonical(path_for_ls + for_match); + fs::canonical(path_for_ls + for_match); fs::path absolute_path = fs::absolute(path_for_ls + for_match); result.push_back(absolute_path.string()); } From fd809956f0f62b721eafc844f6cb65338badede4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 8 Jan 2024 08:14:09 +0000 Subject: [PATCH 084/114] 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 254f7338a308..79a1c3a968b4 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 f38510be868f1966587cd375f1ff1ee01a98b0bb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sat, 30 Dec 2023 20:26:24 +0100 Subject: [PATCH 085/114] Do not fail the CI on an expired token --- tests/ci/get_robot_token.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index 26318b55d159..3781cdc5cc80 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -6,6 +6,7 @@ import boto3 # type: ignore from github import Github from github.AuthenticatedUser import AuthenticatedUser +from github.GithubException import BadCredentialsException from github.NamedUser import NamedUser @@ -68,12 +69,20 @@ def get_best_robot_token(tokens_path: str = "/github-tokens") -> str: } assert tokens - for value in tokens.values(): + for name, value in tokens.items(): gh = Github(value, per_page=100) - # Do not spend additional request to API by accessin user.login unless - # the token is chosen by the remaining requests number - user = gh.get_user() - rest, _ = gh.rate_limiting + try: + # Do not spend additional request to API by accessin user.login unless + # the token is chosen by the remaining requests number + user = gh.get_user() + rest, _ = gh.rate_limiting + except BadCredentialsException: + logging.error( + "The token %(name)s has expired, please update it\n" + "::error::Token %(name)s has expired, it must be updated", + {"name": name}, + ) + continue logging.info("Get token with %s remaining requests", rest) if ROBOT_TOKEN is None: ROBOT_TOKEN = Token(user, value, rest) From 0f2588334f8969caae6c966535cb45625c174730 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 10:20:39 +0000 Subject: [PATCH 086/114] 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 8e73bc8b4840..96be70c55278 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 a84e1ec21757..468d23ed7441 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 284cf5d0e7f4..2be58b5fd2dc 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -396,6 +396,8 @@ class HashJoin : public IJoin 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 @@ class HashJoin : public IJoin /// 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 c1b95560db8f5a666e27f6878cca0c24112a49e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 8 Jan 2024 11:25:57 +0100 Subject: [PATCH 087/114] Add some missing LLVM includes --- src/Common/ConcurrencyControl.h | 7 ++++--- src/Coordination/RaftServerConfig.h | 2 ++ src/Functions/keyvaluepair/impl/NeedleFactory.h | 1 + src/Interpreters/Cache/FileCacheSettings.h | 3 ++- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Common/ConcurrencyControl.h b/src/Common/ConcurrencyControl.h index ba87e3b3a0c6..7e20384aa2a1 100644 --- a/src/Common/ConcurrencyControl.h +++ b/src/Common/ConcurrencyControl.h @@ -1,10 +1,11 @@ #pragma once +#include +#include +#include +#include #include #include -#include -#include -#include namespace DB diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h index 451d61a436e8..0ecbd6464c16 100644 --- a/src/Coordination/RaftServerConfig.h +++ b/src/Coordination/RaftServerConfig.h @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { // default- and copy-constructible version of nuraft::srv_config diff --git a/src/Functions/keyvaluepair/impl/NeedleFactory.h b/src/Functions/keyvaluepair/impl/NeedleFactory.h index b5f0645abcc4..83862a2281a5 100644 --- a/src/Functions/keyvaluepair/impl/NeedleFactory.h +++ b/src/Functions/keyvaluepair/impl/NeedleFactory.h @@ -3,6 +3,7 @@ #include #include +#include #include namespace DB diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index eafd7aafb29d..8fb185038fc2 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -1,8 +1,9 @@ #pragma once +#include +#include #include #include -#include namespace Poco { namespace Util { class AbstractConfiguration; } } // NOLINT(cppcoreguidelines-virtual-class-destructor) From 8dad3f1b22a5f76893f01d0a2cd3dc6d5c6b2427 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 10:58:26 +0000 Subject: [PATCH 088/114] 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 000000000000..8bc1bad225bc --- /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 000000000000..c31ab5e1132d --- /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 089/114] 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 468d23ed7441..b05b10ff25e4 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 090/114] 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 1381e9f28288..987cf4eb1a92 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -434,7 +434,7 @@ class FunctionMakeDateTime64 : public FunctionMakeDateTimeBase }; 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 0ac672ae54dc..1c7d31788e39 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 62784cb9b750..71629ad8dff6 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 091/114] Better clickhouse-format: support VALUES, comments --- .../operations/utilities/clickhouse-format.md | 2 + programs/format/Format.cpp | 147 +++++++++++++----- .../0_stateless/02946_format_values.reference | 139 +++++++++++++++++ .../0_stateless/02946_format_values.sh | 73 +++++++++ 4 files changed, 320 insertions(+), 41 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 3e4295598aad..879bf9d71acb 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 05ba86069d74..f7385335de96 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -30,21 +30,49 @@ #include #include +#include +#include -#pragma GCC diagnostic ignored "-Wunused-function" -#pragma GCC diagnostic ignored "-Wmissing-declarations" - -extern const char * auto_time_zones[]; - +namespace DB::ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} -namespace DB +namespace { -namespace ErrorCodes + +void skipSpacesAndComments(const char*& pos, const char* end, bool print_comments) { -extern const int INVALID_FORMAT_INSERT_QUERY_WITH_DATA; + 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[]; + 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 000000000000..8f896fd3ba05 --- /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 000000000000..cd44d31d1a11 --- /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 092/114] 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 f7385335de96..74306d742f29 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 0aad4d64c555..735d40995340 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 5cdd53b2166c..ba7fe949833f 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 e2d1ec3980ec..2bba75f6788b 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 8b156ffec837..49aa56d6c0a2 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 8f896fd3ba05..90b2a3cb8ef2 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 cd44d31d1a11..36e32de42fa5 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 a891750f535eb2d0482c87d832f91e08a926aac0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 8 Jan 2024 12:08:17 +0000 Subject: [PATCH 093/114] Fix race --- src/Coordination/KeeperServer.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 965e743da393..26ee3668ef61 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -630,6 +630,10 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ const auto preprocess_logs = [&] { auto lock = raft_instance->lockRaft(); + + if (keeper_context->local_logs_preprocessed) + return; + keeper_context->local_logs_preprocessed = true; auto log_store = state_manager->load_log_store(); auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, log_store->next_slot()); From e68fc2df7a3ddd357e80f2f3ea4fecccc62732c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jan 2024 12:16:53 +0000 Subject: [PATCH 094/114] 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 74306d742f29..9f294312ee3c 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 77b349b136d2955e9af1a0e247f666e038c8fb5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jan 2024 13:25:28 +0100 Subject: [PATCH 095/114] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 735094df78b3..76d06ebe837d 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -6,7 +6,6 @@ 01155_rename_move_materialized_view 01214_test_storage_merge_aliases_with_where 01244_optimize_distributed_group_by_sharding_key -01268_mv_scalars 01268_shard_avgweighted 01495_subqueries_in_with_statement 01560_merge_distributed_join @@ -21,21 +20,18 @@ 01925_test_storage_merge_aliases 01952_optimize_distributed_group_by_sharding_key 02174_cte_scalar_cache_mv -02352_grouby_shadows_arg 02354_annoy 02428_parameterized_view -02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number 02575_merge_prewhere_different_default_kind 00917_multiple_joins_denny_crane -00636_partition_key_parts_pruning -02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 02404_memory_bound_merging 02725_agg_projection_resprect_PK 02763_row_policy_storage_merge_alias 02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage -02815_range_dict_no_direct_join # Flaky. Please don't delete them without fixing them: +02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET +02479_race_condition_between_insert_and_droppin_mv 01600_parts_states_metrics_long 01287_max_execution_speed From 7f84b7699326a5614447992f48929b7e0613e023 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jan 2024 14:26:18 +0100 Subject: [PATCH 096/114] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 76d06ebe837d..e8799a1a96ad 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -33,5 +33,3 @@ # Flaky. Please don't delete them without fixing them: 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 02479_race_condition_between_insert_and_droppin_mv -01600_parts_states_metrics_long -01287_max_execution_speed From 2e7a0bfd1c602e7cbb05b8c9ad0bb522ae9636d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jan 2024 14:27:05 +0100 Subject: [PATCH 097/114] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e8799a1a96ad..b03bed2d1961 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -25,11 +25,11 @@ 02493_inconsistent_hex_and_binary_number 02575_merge_prewhere_different_default_kind 00917_multiple_joins_denny_crane -02404_memory_bound_merging 02725_agg_projection_resprect_PK 02763_row_policy_storage_merge_alias 02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage # Flaky. Please don't delete them without fixing them: 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET +02404_memory_bound_merging 02479_race_condition_between_insert_and_droppin_mv From 7abb850757f08dbd6ad9e6f00191116e13b0f565 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jan 2024 14:46:22 +0100 Subject: [PATCH 098/114] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index b03bed2d1961..4643d109c3d4 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -30,6 +30,7 @@ 02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage # Flaky. Please don't delete them without fixing them: +01287_max_execution_speed 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 02404_memory_bound_merging 02479_race_condition_between_insert_and_droppin_mv From 223266321677075e18c727088fe13cdf718d3ada Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 8 Jan 2024 13:24:27 +0100 Subject: [PATCH 099/114] Insignificant typing improvements --- tests/ci/release.py | 3 +-- tests/ci/version_helper.py | 8 +++++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index b5de82b6ca65..50b104ed5296 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -17,12 +17,11 @@ import logging import subprocess -from git_helper import commit, release_branch +from git_helper import Git, commit, release_branch from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, ClickHouseVersion, - Git, VersionType, get_abs_path, get_version_from_repo, diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index fb046e989a91..21d2e3940ce0 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -1,10 +1,10 @@ #!/usr/bin/env python3 import logging import os.path as p -from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter, ArgumentTypeError +from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser, ArgumentTypeError from typing import Any, Dict, List, Literal, Optional, Tuple, Union -from git_helper import TWEAK, Git as Git, get_tags, git_runner, removeprefix +from git_helper import TWEAK, Git, get_tags, git_runner, removeprefix FILE_WITH_VERSION_PATH = "cmake/autogenerated_versions.txt" CHANGELOG_IN_PATH = "debian/changelog.in" @@ -38,6 +38,8 @@ class ClickHouseVersion: """Immutable version class. On update returns a new instance""" + PART_TYPE = Literal["major", "minor", "patch"] + def __init__( self, major: Union[int, str], @@ -60,7 +62,7 @@ def __init__( self._describe = "" self._description = "" - def update(self, part: Literal["major", "minor", "patch"]) -> "ClickHouseVersion": + def update(self, part: PART_TYPE) -> "ClickHouseVersion": """If part is valid, returns a new version""" if part == "major": return self.major_update() From fe19f4597415ff4fd3aebfb5d0a1a4e5e57aa5d9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 8 Jan 2024 13:28:41 +0100 Subject: [PATCH 100/114] Simplify `--type` argument for release.py --- tests/ci/release.py | 55 +++++++++++++++++++++------------------------ 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 50b104ed5296..0f455a63ab51 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -10,12 +10,12 @@ """ -from contextlib import contextmanager -from typing import Any, Iterator, List, Literal, Optional import argparse import json import logging import subprocess +from contextlib import contextmanager +from typing import Any, Final, Iterator, List, Optional, Tuple from git_helper import Git, commit, release_branch from version_helper import ( @@ -60,8 +60,9 @@ def __str__(self): class Release: - BIG = ("major", "minor") - SMALL = ("patch",) + NEW = "new" # type: Final + PATCH = "patch" # type: Final + VALID_TYPE = (NEW, PATCH) # type: Final[Tuple[str, str]] CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) @@ -69,7 +70,7 @@ def __init__( self, repo: Repo, release_commit: str, - release_type: Literal["major", "minor", "patch"], + release_type: str, dry_run: bool, with_stderr: bool, ): @@ -78,7 +79,7 @@ def __init__( self.release_commit = release_commit self.dry_run = dry_run self.with_stderr = with_stderr - assert release_type in self.BIG + self.SMALL + assert release_type in self.VALID_TYPE self.release_type = release_type self._git = Git() self._version = get_version_from_repo(git=self._git) @@ -193,16 +194,7 @@ def do( if check_branch: self.check_branch() - if self.release_type in self.BIG: - if self._version.minor >= 12 and self.release_type != "major": - raise ValueError( - "The release type must be 'major' for minor versions>=12" - ) - if self._version.minor < 12 and self.release_type == "major": - raise ValueError( - "The release type must be 'minor' for minor versions<12" - ) - + if self.release_type == self.NEW: with self._checkout(self.release_commit, True): # Checkout to the commit, it will provide the correct current version with self.testing(): @@ -213,7 +205,7 @@ def do( self.release_commit, ) - elif self.release_type in self.SMALL: + elif self.release_type == self.PATCH: with self._checkout(self.release_commit, True): with self.stable(): logging.info( @@ -243,15 +235,12 @@ def check_no_tags_after(self): def check_branch(self): branch = self.release_branch - if self.release_type in self.BIG: + if self.release_type == self.NEW: # Commit to spin up the release must belong to a main branch branch = "master" - elif self.release_type not in self.SMALL: + elif self.release_type != self.PATCH: raise ( - ValueError( - f"release_type {self.release_type} neither in {self.BIG} nor " - f"in {self.SMALL}" - ) + ValueError(f"release_type {self.release_type} not in {self.VALID_TYPE}") ) # Prefetch the branch to have it updated @@ -295,6 +284,14 @@ def _commit_cmake_contributors( dry_run=self.dry_run, ) + @property + def bump_part(self) -> ClickHouseVersion.PART_TYPE: + if self.release_type == Release.NEW: + if self._version.minor >= 12: + return "major" + return "minor" + return "patch" + @property def has_rollback(self) -> bool: return bool(self._rollback_stack) @@ -334,7 +331,7 @@ def stable(self): version_type = self.get_stable_release_type() self.version.with_description(version_type) with self._create_gh_release(False): - self.version = self.version.update(self.release_type) + self.version = self.version.update(self.bump_part) self.version.with_description(version_type) self._update_cmake_contributors(self.version) # Checking out the commit of the branch and not the branch itself, @@ -357,7 +354,7 @@ def stable(self): def testing(self): # Create branch for a version bump self.read_version() - self.version = self.version.update(self.release_type) + self.version = self.version.update(self.bump_part) helper_branch = f"{self.version.major}.{self.version.minor}-prepare" with self._create_branch(helper_branch, self.release_commit): with self._checkout(helper_branch, True): @@ -433,7 +430,7 @@ def _bump_release_branch(self): @contextmanager def _bump_testing_version(self, helper_branch: str) -> Iterator[None]: self.read_version() - self.version = self.version.update(self.release_type) + self.version = self.version.update(self.bump_part) self.version.with_description(VersionType.TESTING) self._update_cmake_contributors(self.version) self._commit_cmake_contributors(self.version) @@ -597,10 +594,10 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--type", required=True, - choices=Release.BIG + Release.SMALL, + choices=Release.VALID_TYPE, dest="release_type", help="a release type to bump the major.minor.patch version part, " - "new branch is created only for 'major' and 'minor'", + "new branch is created only for the value 'new'", ) parser.add_argument("--with-release-branch", default=True, help=argparse.SUPPRESS) parser.add_argument("--check-dirty", default=True, help=argparse.SUPPRESS) @@ -626,7 +623,7 @@ def parse_args() -> argparse.Namespace: action="store_false", default=argparse.SUPPRESS, help="(debug or development only, dangerous) if set, skip the branch check for " - "a run. By default, 'major' and 'minor' types work only for master, and 'patch' " + "a run. By default, 'new' type work only for master, and 'patch' " "works only for a release branches, that name " "should be the same as '$MAJOR.$MINOR' version, e.g. 22.2", ) From a2a3783366e3a38ebc2392fe963270633a994d12 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 8 Jan 2024 13:33:49 +0100 Subject: [PATCH 101/114] Small renaming to reflect the current release process --- tests/ci/release.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 0f455a63ab51..0076c6ab985b 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -197,7 +197,7 @@ def do( if self.release_type == self.NEW: with self._checkout(self.release_commit, True): # Checkout to the commit, it will provide the correct current version - with self.testing(): + with self.new_release(): with self.create_release_branch(): logging.info( "Publishing release %s from commit %s is done", @@ -207,7 +207,7 @@ def do( elif self.release_type == self.PATCH: with self._checkout(self.release_commit, True): - with self.stable(): + with self.patch_release(): logging.info( "Publishing release %s from commit %s is done", self.release_version.describe, @@ -325,7 +325,7 @@ def create_release_branch(self): yield @contextmanager - def stable(self): + def patch_release(self): self.check_no_tags_after() self.read_version() version_type = self.get_stable_release_type() @@ -351,14 +351,14 @@ def stable(self): yield @contextmanager - def testing(self): + def new_release(self): # Create branch for a version bump self.read_version() self.version = self.version.update(self.bump_part) helper_branch = f"{self.version.major}.{self.version.minor}-prepare" with self._create_branch(helper_branch, self.release_commit): with self._checkout(helper_branch, True): - with self._bump_testing_version(helper_branch): + with self._bump_version_in_master(helper_branch): yield @property @@ -428,7 +428,7 @@ def _bump_release_branch(self): yield @contextmanager - def _bump_testing_version(self, helper_branch: str) -> Iterator[None]: + def _bump_version_in_master(self, helper_branch: str) -> Iterator[None]: self.read_version() self.version = self.version.update(self.bump_part) self.version.with_description(VersionType.TESTING) @@ -443,7 +443,7 @@ def _bump_testing_version(self, helper_branch: str) -> Iterator[None]: "--label 'do not test' --assignee @me", dry_run=self.dry_run, ) - # Here the testing part is done + # Here the new release part is done yield @contextmanager From 9bd48512238c0eb7d8f2bfbaaaf674e31525a669 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jan 2024 15:35:02 +0100 Subject: [PATCH 102/114] 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 a97879eaca89..ffb3f461f1ca 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 103/114] 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 5495d58d65f0..f047c21b470b 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 @@ class RandomStringPrepareWalker : public Regexp::Walker 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 65bfe0c92089..68b22404d5ad 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 c970b193adbd..7b4507a3abcf 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 67ccf1ffe84e..2f6260891c61 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 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 104/114] 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 ffb3f461f1ca..babcecf70045 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 105/114] 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 21235914f7c1..fe513199ac2f 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 b2434d0c10586f90936d9a7352395fc3a4afef1a Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Mon, 8 Jan 2024 10:49:09 -0800 Subject: [PATCH 106/114] minor refactoring and cosmetic changes --- .../functions/time-series-functions.md | 2 +- src/Functions/seriesDecomposeSTL.cpp | 21 +++++++------------ 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 59c261df8068..144d832b36a7 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -71,7 +71,7 @@ seriesDecomposeSTL(series, period); **Arguments** - `series` - An array of numeric values -- `period` - A positive number +- `period` - A positive integer The number of data points in `series` should be at least twice the value of `period`. diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 2739c7b71779..21e36761213d 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -11,7 +11,6 @@ #pragma clang diagnostic pop #endif -#include #include #include #include @@ -20,7 +19,6 @@ #include #include #include -#include namespace DB @@ -31,9 +29,7 @@ extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } -/* -Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) -*/ +// Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess) class FunctionSeriesDecomposeSTL : public IFunction { public: @@ -78,8 +74,6 @@ class FunctionSeriesDecomposeSTL : public IFunction const IColumn & src_data = array->getData(); const ColumnArray::Offsets & src_offsets = array->getOffsets(); - UInt64 period; - auto res = ColumnFloat32::create(); auto & res_data = res->getData(); @@ -93,6 +87,7 @@ class FunctionSeriesDecomposeSTL : public IFunction for (size_t i = 0; i < src_offsets.size(); ++i) { + UInt64 period; auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); if (checkAndGetColumn(period_ptr.get()) || checkAndGetColumn(period_ptr.get()) @@ -124,13 +119,13 @@ class FunctionSeriesDecomposeSTL : public IFunction || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue) || executeNumber(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)) { - res_data.insert(res_data.end(), seasonal.begin(), seasonal.end()); + res_data.insert(seasonal.begin(), seasonal.end()); res_col_offsets_data.push_back(res_data.size()); - res_data.insert(res_data.end(), trend.begin(), trend.end()); + res_data.insert(trend.begin(), trend.end()); res_col_offsets_data.push_back(res_data.size()); - res_data.insert(res_data.end(), residue.begin(), residue.end()); + res_data.insert(residue.begin(), residue.end()); res_col_offsets_data.push_back(res_data.size()); root_offsets_data.push_back(res_col_offsets->size()); @@ -152,8 +147,8 @@ class FunctionSeriesDecomposeSTL : public IFunction bool executeNumber( const IColumn & src_data, UInt64 period, - ColumnArray::Offset & start, - ColumnArray::Offset & end, + ColumnArray::Offset start, + ColumnArray::Offset end, std::vector & seasonal, std::vector & trend, std::vector & residue) const @@ -174,7 +169,7 @@ class FunctionSeriesDecomposeSTL : public IFunction std::vector src(src_vec.begin() + start, src_vec.begin() + end); - auto res = stl::params().fit(src, static_cast(period)); + auto res = stl::params().fit(src, period); if (res.seasonal.empty()) return false; From b05211cc342c6ba329f1699c18fc2923848cdb3d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 8 Jan 2024 20:16:21 +0100 Subject: [PATCH 107/114] Update 02947_dropped_tables_parts.sql --- tests/queries/0_stateless/02947_dropped_tables_parts.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02947_dropped_tables_parts.sql b/tests/queries/0_stateless/02947_dropped_tables_parts.sql index 265862b448c2..554a19ca6b11 100644 --- a/tests/queries/0_stateless/02947_dropped_tables_parts.sql +++ b/tests/queries/0_stateless/02947_dropped_tables_parts.sql @@ -8,6 +8,7 @@ INSERT INTO 02947_table_1 VALUES (1),(2); INSERT INTO 02947_table_2 VALUES (3),(4); SELECT database, table, name FROM system.parts WHERE database = currentDatabase() AND startsWith(table, '02947_table_'); +select * from system.dropped_tables_parts format Null; DROP TABLE 02947_table_1; DROP TABLE 02947_table_2; From ee37f551a249f6264b14fba2947442eb11e6009e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Date: Mon, 8 Jan 2024 22:33:48 -0800 Subject: [PATCH 108/114] independent registration of interpreters (#58443) * make interpreter factory an instance * add registerInterpreter * register interpreters individually * try wire everything up * fix style * fix test --- programs/copier/ClusterCopier.cpp | 6 +- programs/copier/ClusterCopierApp.cpp | 2 + programs/format/Format.cpp | 2 + programs/local/LocalServer.cpp | 2 + programs/server/Server.cpp | 2 + .../Access/InterpreterCreateQuotaQuery.cpp | 10 + .../Access/InterpreterCreateRoleQuery.cpp | 11 + .../InterpreterCreateRowPolicyQuery.cpp | 10 + .../InterpreterCreateSettingsProfileQuery.cpp | 11 + .../Access/InterpreterCreateUserQuery.cpp | 10 + .../InterpreterDropAccessEntityQuery.cpp | 10 + .../Access/InterpreterGrantQuery.cpp | 10 + .../InterpreterMoveAccessEntityQuery.cpp | 10 + .../Access/InterpreterSetRoleQuery.cpp | 10 + .../InterpreterShowAccessEntitiesQuery.cpp | 10 + .../Access/InterpreterShowAccessQuery.cpp | 10 + ...InterpreterShowCreateAccessEntityQuery.cpp | 11 + .../Access/InterpreterShowGrantsQuery.cpp | 10 + .../Access/InterpreterShowPrivilegesQuery.cpp | 11 + .../InterpreterAlterNamedCollectionQuery.cpp | 10 + src/Interpreters/InterpreterAlterQuery.cpp | 10 + src/Interpreters/InterpreterBackupQuery.cpp | 10 + src/Interpreters/InterpreterCheckQuery.cpp | 10 + .../InterpreterCreateFunctionQuery.cpp | 10 + .../InterpreterCreateIndexQuery.cpp | 10 + .../InterpreterCreateNamedCollectionQuery.cpp | 10 + src/Interpreters/InterpreterCreateQuery.cpp | 10 + src/Interpreters/InterpreterDeleteQuery.cpp | 10 + .../InterpreterDescribeCacheQuery.cpp | 10 + src/Interpreters/InterpreterDescribeQuery.cpp | 10 + .../InterpreterDropFunctionQuery.cpp | 10 + .../InterpreterDropIndexQuery.cpp | 10 + .../InterpreterDropNamedCollectionQuery.cpp | 10 + src/Interpreters/InterpreterDropQuery.cpp | 9 + src/Interpreters/InterpreterExistsQuery.cpp | 9 + src/Interpreters/InterpreterExplainQuery.cpp | 10 + .../InterpreterExternalDDLQuery.cpp | 10 + src/Interpreters/InterpreterFactory.cpp | 230 +++++++----------- src/Interpreters/InterpreterFactory.h | 25 +- src/Interpreters/InterpreterInsertQuery.cpp | 9 + .../InterpreterKillQueryQuery.cpp | 10 + src/Interpreters/InterpreterOptimizeQuery.cpp | 9 + src/Interpreters/InterpreterRenameQuery.cpp | 10 + .../InterpreterSelectIntersectExceptQuery.cpp | 10 + src/Interpreters/InterpreterSelectQuery.cpp | 9 + .../InterpreterSelectQueryAnalyzer.cpp | 10 + .../InterpreterSelectWithUnionQuery.cpp | 10 + src/Interpreters/InterpreterSetQuery.cpp | 9 + .../InterpreterShowColumnsQuery.cpp | 9 + .../InterpreterShowCreateQuery.cpp | 10 + .../InterpreterShowEngineQuery.cpp | 10 + .../InterpreterShowFunctionsQuery.cpp | 10 + .../InterpreterShowIndexesQuery.cpp | 9 + .../InterpreterShowProcesslistQuery.cpp | 10 + .../InterpreterShowSettingQuery.cpp | 9 + .../InterpreterShowTablesQuery.cpp | 11 + src/Interpreters/InterpreterSystemQuery.cpp | 10 + .../InterpreterTransactionControlQuery.cpp | 10 + src/Interpreters/InterpreterUndropQuery.cpp | 10 + src/Interpreters/InterpreterUseQuery.cpp | 10 + src/Interpreters/InterpreterWatchQuery.cpp | 10 + src/Interpreters/executeQuery.cpp | 2 +- .../fuzzers/execute_query_fuzzer.cpp | 2 + src/Interpreters/registerInterpreters.cpp | 120 +++++++++ src/Interpreters/registerInterpreters.h | 6 + 65 files changed, 787 insertions(+), 148 deletions(-) create mode 100644 src/Interpreters/registerInterpreters.cpp create mode 100644 src/Interpreters/registerInterpreters.h diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 1dfdcb3c745b..59505d08f5c2 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1559,7 +1559,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( QueryPipeline input; QueryPipeline output; { - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); + BlockIO io_insert = InterpreterFactory::instance().get(query_insert_ast, context_insert)->execute(); InterpreterSelectWithUnionQuery select(query_select_ast, context_select, SelectQueryOptions{}); QueryPlan plan; @@ -1944,7 +1944,7 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); - auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline; + auto pipeline = InterpreterFactory::instance().get(query_ast, local_context)->execute().pipeline; PullingPipelineExecutor executor(pipeline); Block block; executor.pull(block); @@ -1989,7 +1989,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); - auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline; + auto pipeline = InterpreterFactory::instance().get(query_ast, local_context)->execute().pipeline; PullingPipelineExecutor executor(pipeline); Block result; executor.pull(result); diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index e3371185aad2..53f798885731 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -157,6 +158,7 @@ void ClusterCopierApp::mainImpl() context->setApplicationType(Context::ApplicationType::LOCAL); context->setPath(process_path + "/"); + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 05ba86069d74..ecf02283ab74 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -128,6 +129,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) hash_func.update(options["seed"].as()); } + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ccd3d84630f5..c9841277b6d3 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -486,6 +487,7 @@ try Poco::ErrorHandler::set(&error_handler); } + registerInterpreters(); /// Don't initialize DateLUT registerFunctions(); registerAggregateFunctions(); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1fa3d1cfa738..80d4b42890db 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -646,6 +647,7 @@ try } #endif + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index b62f3a8b0bd3..56608644425a 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -160,4 +161,13 @@ void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTC updateQuotaFromQueryImpl(quota, query, {}, {}); } +void registerInterpreterCreateQuotaQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateQuotaQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index fef1f285c8b6..4936bd152622 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -122,4 +123,14 @@ void InterpreterCreateRoleQuery::updateRoleFromQuery(Role & role, const ASTCreat { updateRoleFromQueryImpl(role, query, {}, {}); } + +void registerInterpreterCreateRoleQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateRoleQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index e4593222f6d4..b48c3880c592 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -148,4 +149,13 @@ AccessRightsElements InterpreterCreateRowPolicyQuery::getRequiredAccess() const return res; } +void registerInterpreterCreateRowPolicyQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateRowPolicyQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 3a96c0a96ff1..029deff9b225 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -138,4 +139,14 @@ void InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(Setti { updateSettingsProfileFromQueryImpl(SettingsProfile, query, {}, {}, {}); } + +void registerInterpreterCreateSettingsProfileQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateSettingsProfileQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 00e21f27d2ee..32c51b745c79 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -261,4 +262,13 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true); } +void registerInterpreterCreateUserQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateUserQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 371ed248306a..612000c3480f 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -95,4 +96,13 @@ AccessRightsElements InterpreterDropAccessEntityQuery::getRequiredAccess() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); } +void registerInterpreterDropAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 259c6b395249..0f2d65abb5e6 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -480,4 +481,13 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery updateFromQuery(role, query); } +void registerInterpreterGrantQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterGrantQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index 49e90783a59f..a05fc86f8669 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -90,4 +91,13 @@ AccessRightsElements InterpreterMoveAccessEntityQuery::getRequiredAccess() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); } +void registerInterpreterMoveAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterMoveAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 69740f90d4cc..244679235424 100644 --- a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -90,4 +91,13 @@ void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const Roles user.default_roles = roles_from_query; } +void registerInterpreterSetRoleQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSetRoleQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index bffb47ac7140..76979ed86c8b 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -125,4 +126,13 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const (order.empty() ? "" : " ORDER BY " + order); } +void registerInterpreterShowAccessEntitiesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowAccessEntitiesQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index e9862e993932..23ce66fe6634 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -80,4 +81,13 @@ ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const return result; } +void registerInterpreterShowAccessQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowAccessQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index ec2e60b2ef71..a55588baeaa4 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -420,4 +421,14 @@ AccessRightsElements InterpreterShowCreateAccessEntityQuery::getRequiredAccess() } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by SHOW CREATE query", toString(show_query.type)); } + +void registerInterpreterShowCreateAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowCreateAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 56fbb34a577f..ba96dafbf0dd 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -198,4 +199,13 @@ ASTs InterpreterShowGrantsQuery::getAttachGrantQueries(const IAccessEntity & use return getGrantQueriesImpl(user_or_role, nullptr, true); } +void registerInterpreterShowGrantsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowGrantsQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp index 1a0b441a06d3..42b7fc510227 100644 --- a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -15,4 +16,14 @@ BlockIO InterpreterShowPrivilegesQuery::execute() return executeQuery("SELECT * FROM system.privileges", context, QueryFlags{ .internal = true }).second; } +void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowPrivilegesQuery", create_fn); +} + + } diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 478735c432b0..a4e868795962 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterAlterNamedCollectionQuery::execute() return {}; } +void registerInterpreterAlterNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterAlterNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 2a34932d950c..bfcb0d6dd396 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -535,4 +536,13 @@ void InterpreterAlterQuery::extendQueryLogElemImpl(QueryLogElement & elem, const } } +void registerInterpreterAlterQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterAlterQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index be5fcedce271..6f76b21a7b8d 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -47,4 +48,13 @@ BlockIO InterpreterBackupQuery::execute() return res_io; } +void registerInterpreterBackupQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterBackupQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 6f28aee5f13c..0cc4afd62f22 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -472,4 +473,13 @@ BlockIO InterpreterCheckQuery::execute() return res; } +void registerInterpreterCheckQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCheckQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index ea59115b0772..18e9ba4a64b2 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -53,4 +54,13 @@ BlockIO InterpreterCreateFunctionQuery::execute() return {}; } +void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateFunctionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index ed29c82a0f0b..cd2f996c74c0 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -99,4 +100,13 @@ BlockIO InterpreterCreateIndexQuery::execute() return {}; } +void registerInterpreterCreateIndexQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateIndexQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 3b0fba5fd9fa..41e87bb73dda 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterCreateNamedCollectionQuery::execute() return {}; } +void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 29abe2929081..2bddb4935deb 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -1895,4 +1896,13 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } +void registerInterpreterCreateQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 52723416400a..5c13a1145d1c 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -110,4 +111,13 @@ BlockIO InterpreterDeleteQuery::execute() } } +void registerInterpreterDeleteQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDeleteQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index 54b43a8850b9..c7e863bf260b 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -68,4 +69,13 @@ BlockIO InterpreterDescribeCacheQuery::execute() return res; } +void registerInterpreterDescribeCacheQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDescribeCacheQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 755de7e4c5f5..1aab72afcc1d 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -271,4 +272,13 @@ void InterpreterDescribeQuery::addSubcolumns(const ColumnDescription & column, b }, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type)); } +void registerInterpreterDescribeQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDescribeQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index c2cd24044da2..2661fd9058c2 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -49,4 +50,13 @@ BlockIO InterpreterDropFunctionQuery::execute() return {}; } +void registerInterpreterDropFunctionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropFunctionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropIndexQuery.cpp b/src/Interpreters/InterpreterDropIndexQuery.cpp index 98d489424874..025677eeb91e 100644 --- a/src/Interpreters/InterpreterDropIndexQuery.cpp +++ b/src/Interpreters/InterpreterDropIndexQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -68,4 +69,13 @@ BlockIO InterpreterDropIndexQuery::execute() return {}; } +void registerInterpreterDropIndexQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropIndexQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index fe49b1cfd7c8..baadc85f443e 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterDropNamedCollectionQuery::execute() return {}; } +void registerInterpreterDropNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b8c9d5dabb54..711100b5de1e 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -519,4 +520,12 @@ bool InterpreterDropQuery::supportsTransactions() const && drop.table; } +void registerInterpreterDropQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 90fa15bf63f6..e4176a445334 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -81,4 +82,12 @@ QueryPipeline InterpreterExistsQuery::executeImpl() "result" }})); } +void registerInterpreterExistsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExistsQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4c70dcb3cc3f..458be843b59e 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -607,4 +608,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() return QueryPipeline(std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } +void registerInterpreterExplainQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExplainQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index c0acb1e03ebd..e68ff18dbb69 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -1,6 +1,7 @@ #include "config.h" #include +#include #include #include @@ -67,4 +68,13 @@ BlockIO InterpreterExternalDDLQuery::execute() return BlockIO(); } +void registerInterpreterExternalDDLQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExternalDDLQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e32cbe4ccad0..84432415f5e4 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include @@ -50,68 +49,14 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - #include - -#include #include #include #include @@ -131,10 +76,22 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int LOGICAL_ERROR; } +InterpreterFactory & InterpreterFactory::instance() +{ + static InterpreterFactory interpreter_fact; + return interpreter_fact; +} + +void InterpreterFactory::registerInterpreter(const std::string & name, CreatorFn creator_fn) +{ + if (!interpreters.emplace(name, std::move(creator_fn)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "InterpreterFactory: the interpreter name '{}' is not unique", name); +} -std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) +InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { ProfileEvents::increment(ProfileEvents::Query); @@ -147,258 +104,247 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); } + Arguments arguments { + .query = query, + .context = context, + .options = options + }; + + String interpreter_name; + if (query->as()) { if (context->getSettingsRef().allow_experimental_analyzer) - return std::make_unique(query, context, options); - + interpreter_name = "InterpreterSelectQueryAnalyzer"; /// This is internal part of ASTSelectWithUnionQuery. /// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child. - return std::make_unique(query, context, options); + else + interpreter_name = "InterpreterSelectQuery"; } else if (query->as()) { ProfileEvents::increment(ProfileEvents::SelectQuery); if (context->getSettingsRef().allow_experimental_analyzer) - return std::make_unique(query, context, options); - - return std::make_unique(query, context, options); + interpreter_name = "InterpreterSelectQueryAnalyzer"; + else + interpreter_name = "InterpreterSelectWithUnionQuery"; } else if (query->as()) { - return std::make_unique(query, context, options); + interpreter_name = "InterpreterSelectIntersectExceptQuery"; } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); bool allow_materialized = static_cast(context->getSettingsRef().insert_allow_materialized_columns); - return std::make_unique(query, context, allow_materialized); + arguments.allow_materialized = allow_materialized; + interpreter_name = "InterpreterInsertQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterUndropQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterRenameQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowTablesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowColumnsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowIndexesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowSettingQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowEnginesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowFunctionsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterUseQuery"; } else if (query->as()) { /// readonly is checked inside InterpreterSetQuery - return std::make_unique(query, context); + interpreter_name = "InterpreterSetQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterSetRoleQuery"; } else if (query->as()) { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); + interpreter_name = "InterpreterOptimizeQuery"; } - else if (query->as()) + else if (query->as() || query->as() || query->as() || query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExistsQuery"; } - else if (query->as()) + else if (query->as() || query->as() || query->as() || query->as()) { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowCreateQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDescribeQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDescribeCacheQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExplainQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowProcesslistQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterAlterQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterAlterNamedCollectionQuery"; } else if (query->as() || query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCheckQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterKillQueryQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterSystemQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterWatchQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateUserQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateRoleQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateQuotaQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateRowPolicyQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateSettingsProfileQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterMoveAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropNamedCollectionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterGrantQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowCreateAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowGrantsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowAccessEntitiesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name= "InterpreterShowAccessQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowPrivilegesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExternalDDLQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterTransactionControlQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateFunctionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropFunctionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateIndexQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateNamedCollectionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropIndexQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterBackupQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDeleteQuery"; } - else - { + + if (!interpreters.contains(interpreter_name)) throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Unknown type of query: {}", query->getID()); - } + + // creator_fn creates and returns a InterpreterPtr with the supplied arguments + auto creator_fn = interpreters.at(interpreter_name); + + return creator_fn(arguments); } } diff --git a/src/Interpreters/InterpreterFactory.h b/src/Interpreters/InterpreterFactory.h index 774cbd1cb0fb..3cf3b02d826b 100644 --- a/src/Interpreters/InterpreterFactory.h +++ b/src/Interpreters/InterpreterFactory.h @@ -11,13 +11,34 @@ namespace DB class Context; -class InterpreterFactory +class InterpreterFactory : private boost::noncopyable { public: - static std::unique_ptr get( + static InterpreterFactory & instance(); + + struct Arguments + { + ASTPtr & query; + ContextMutablePtr context; + const SelectQueryOptions & options; + bool allow_materialized; + }; + + using InterpreterPtr = std::unique_ptr; + + InterpreterPtr get( ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options = {}); + + using CreatorFn = std::function; + + using Interpreters = std::unordered_map; + + void registerInterpreter(const std::string & name, CreatorFn creator_fn); + +private: + Interpreters interpreters; }; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 283289f0dfc5..c8e05fcd5e31 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -680,4 +681,12 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons extendQueryLogElemImpl(elem, context_); } +void registerInterpreterInsertQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.allow_materialized); + }; + factory.registerInterpreter("InterpreterInsertQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 6e1422f29389..5efffdaa194a 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -452,4 +453,13 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster return required_access; } +void registerInterpreterKillQueryQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterKillQueryQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index ae456e8b31de..1e2eaa50ab1e 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -93,4 +94,12 @@ AccessRightsElements InterpreterOptimizeQuery::getRequiredAccess() const return required_access; } +void registerInterpreterOptimizeQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterOptimizeQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index ae79b3f932ec..52001fdcaf4a 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -228,4 +229,13 @@ void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons } } +void registerInterpreterRenameQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterRenameQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 2218ed4417bf..6eac2db20c98 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -210,4 +211,13 @@ void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogEleme } } +void registerInterpreterSelectIntersectExceptQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectIntersectExceptQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e8482ccbd74..e057b4fd7e08 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include #include @@ -3333,5 +3334,13 @@ bool InterpreterSelectQuery::isQueryWithFinal(const SelectQueryInfo & info) return result; } +void registerInterpreterSelectQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index eed9d03ab5a6..f498fc7ec85d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -267,4 +268,13 @@ void InterpreterSelectQueryAnalyzer::extendQueryLogElemImpl(QueryLogElement & el elem.used_row_policies.emplace(used_row_policy); } +void registerInterpreterSelectQueryAnalyzer(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectQueryAnalyzer", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 2ae74955e4f8..16bc4b1fe2e7 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -410,4 +411,13 @@ void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & e } } +void registerInterpreterSelectWithUnionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectWithUnionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 2c0baa0d4b36..261c781e0baa 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -91,4 +92,12 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta } } +void registerInterpreterSetQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSetQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index a5b223874487..149ba6d75757 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -164,5 +165,13 @@ BlockIO InterpreterShowColumnsQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowColumnsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowColumnsQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 0d60f13af663..9edac1fd8e19 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -104,4 +105,13 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() "statement"}})); } +void registerInterpreterShowCreateQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + + factory.registerInterpreter("InterpreterShowCreateQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowEngineQuery.cpp b/src/Interpreters/InterpreterShowEngineQuery.cpp index 2927fbd0f2db..f2d057a3fcf9 100644 --- a/src/Interpreters/InterpreterShowEngineQuery.cpp +++ b/src/Interpreters/InterpreterShowEngineQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,4 +16,13 @@ BlockIO InterpreterShowEnginesQuery::execute() return executeQuery("SELECT * FROM system.table_engines ORDER BY name", getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowEnginesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowEnginesQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index a9da01b09883..e83f61eac53f 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -43,4 +44,13 @@ FROM {}.{})", return rewritten_query; } +void registerInterpreterShowFunctionsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowFunctionsQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 09b70e951db1..e8005ead91e1 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -104,5 +105,13 @@ BlockIO InterpreterShowIndexesQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowIndexesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowIndexesQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index f711cc0dac99..7bdb94482da1 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,4 +16,13 @@ BlockIO InterpreterShowProcesslistQuery::execute() return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowProcesslistQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowProcesslistQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowSettingQuery.cpp b/src/Interpreters/InterpreterShowSettingQuery.cpp index 45e9b8a1f1cb..90acaa7b083c 100644 --- a/src/Interpreters/InterpreterShowSettingQuery.cpp +++ b/src/Interpreters/InterpreterShowSettingQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -29,5 +30,13 @@ BlockIO InterpreterShowSettingQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowSettingQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowSettingQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 0ca6578128d6..51038aaca46b 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -221,4 +222,14 @@ BlockIO InterpreterShowTablesQuery::execute() /// sort the output of SHOW otherwise (SELECT * FROM (SHOW ...) ORDER BY ...) is rejected) and 3. some /// SQL tests can take advantage of this. + +void registerInterpreterShowTablesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowTablesQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index db02ee13a4f9..8d3ed0954057 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -1378,4 +1379,13 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() return required_access; } +void registerInterpreterSystemQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSystemQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index a0a82121ba86..b1758013f183 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -135,4 +136,13 @@ BlockIO InterpreterTransactionControlQuery::executeSetSnapshot(ContextMutablePtr return {}; } +void registerInterpreterTransactionControlQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterTransactionControlQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index bdd72b6d3ea6..8401c47df6b2 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -72,4 +73,13 @@ AccessRightsElements InterpreterUndropQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::UNDROP_TABLE, undrop.getDatabase(), undrop.getTable()); return required_access; } + +void registerInterpreterUndropQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterUndropQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index b71f3a9cc1c0..58be12927b9b 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -16,4 +17,13 @@ BlockIO InterpreterUseQuery::execute() return {}; } +void registerInterpreterUseQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterUseQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 8865c47a7852..2b68c5d7a10a 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -12,6 +12,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -103,4 +104,13 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline() return pipeline; } +void registerInterpreterWatchQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterWatchQuery", create_fn); +} + } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 63804d2d86f2..27986df11859 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1043,7 +1043,7 @@ static std::tuple executeQueryImpl( } } - interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); + interpreter = InterpreterFactory::instance().get(ast, context, SelectQueryOptions(stage).setInternal(internal)); const auto & query_settings = context->getSettingsRef(); if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index fd023754abf5..6f84a60f2afe 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -1,5 +1,6 @@ #include #include +#include #include "Processors/Executors/PullingPipelineExecutor.h" #include @@ -29,6 +30,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp new file mode 100644 index 000000000000..481d0597a854 --- /dev/null +++ b/src/Interpreters/registerInterpreters.cpp @@ -0,0 +1,120 @@ +#include + +namespace DB +{ + +void registerInterpreterSelectQuery(InterpreterFactory & factory); +void registerInterpreterSelectQueryAnalyzer(InterpreterFactory & factory); +void registerInterpreterSelectWithUnionQuery(InterpreterFactory & factory); +void registerInterpreterSelectIntersectExceptQuery(InterpreterFactory & factory); +void registerInterpreterInsertQuery(InterpreterFactory & factory); +void registerInterpreterCreateQuery(InterpreterFactory & factory); +void registerInterpreterDropQuery(InterpreterFactory & factory); +void registerInterpreterUndropQuery(InterpreterFactory & factory); +void registerInterpreterRenameQuery(InterpreterFactory & factory); +void registerInterpreterShowTablesQuery(InterpreterFactory & factory); +void registerInterpreterShowColumnsQuery(InterpreterFactory & factory); +void registerInterpreterShowIndexesQuery(InterpreterFactory & factory); +void registerInterpreterShowSettingQuery(InterpreterFactory & factory); +void registerInterpreterShowEnginesQuery(InterpreterFactory & factory); +void registerInterpreterShowFunctionsQuery(InterpreterFactory & factory); +void registerInterpreterUseQuery(InterpreterFactory & factory); +void registerInterpreterSetQuery(InterpreterFactory & factory); +void registerInterpreterSetRoleQuery(InterpreterFactory & factory); +void registerInterpreterOptimizeQuery(InterpreterFactory & factory); +void registerInterpreterExistsQuery(InterpreterFactory & factory); +void registerInterpreterShowCreateQuery(InterpreterFactory & factory); +void registerInterpreterDescribeQuery(InterpreterFactory & factory); +void registerInterpreterDescribeCacheQuery(InterpreterFactory & factory); +void registerInterpreterExplainQuery(InterpreterFactory & factory); +void registerInterpreterShowProcesslistQuery(InterpreterFactory & factory); +void registerInterpreterAlterQuery(InterpreterFactory & factory); +void registerInterpreterAlterNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterCheckQuery(InterpreterFactory & factory); +void registerInterpreterKillQueryQuery(InterpreterFactory & factory); +void registerInterpreterSystemQuery(InterpreterFactory & factory); +void registerInterpreterWatchQuery(InterpreterFactory & factory); +void registerInterpreterCreateUserQuery(InterpreterFactory & factory); +void registerInterpreterCreateRoleQuery(InterpreterFactory & factory); +void registerInterpreterCreateQuotaQuery(InterpreterFactory & factory); +void registerInterpreterCreateRowPolicyQuery(InterpreterFactory & factory); +void registerInterpreterCreateSettingsProfileQuery(InterpreterFactory & factory); +void registerInterpreterDropAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterMoveAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterDropNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterGrantQuery(InterpreterFactory & factory); +void registerInterpreterShowCreateAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterShowGrantsQuery(InterpreterFactory & factory); +void registerInterpreterShowAccessEntitiesQuery(InterpreterFactory & factory); +void registerInterpreterShowAccessQuery(InterpreterFactory & factory); +void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory); +void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); +void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); +void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); +void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); +void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); +void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterDropIndexQuery(InterpreterFactory & factory); +void registerInterpreterBackupQuery(InterpreterFactory & factory); +void registerInterpreterDeleteQuery(InterpreterFactory & factory); + +void registerInterpreters() +{ + auto & factory = InterpreterFactory::instance(); + + registerInterpreterSelectQuery(factory); + registerInterpreterSelectQueryAnalyzer(factory); + registerInterpreterSelectWithUnionQuery(factory); + registerInterpreterSelectIntersectExceptQuery(factory); + registerInterpreterInsertQuery(factory); + registerInterpreterCreateQuery(factory); + registerInterpreterDropQuery(factory); + registerInterpreterUndropQuery(factory); + registerInterpreterRenameQuery(factory); + registerInterpreterShowTablesQuery(factory); + registerInterpreterShowColumnsQuery(factory); + registerInterpreterShowIndexesQuery(factory); + registerInterpreterShowSettingQuery(factory); + registerInterpreterShowEnginesQuery(factory); + registerInterpreterShowFunctionsQuery(factory); + registerInterpreterUseQuery(factory); + registerInterpreterSetQuery(factory); + registerInterpreterSetRoleQuery(factory); + registerInterpreterOptimizeQuery(factory); + registerInterpreterExistsQuery(factory); + registerInterpreterShowCreateQuery(factory); + registerInterpreterDescribeQuery(factory); + registerInterpreterDescribeCacheQuery(factory); + registerInterpreterExplainQuery(factory); + registerInterpreterShowProcesslistQuery(factory); + registerInterpreterAlterQuery(factory); + registerInterpreterAlterNamedCollectionQuery(factory); + registerInterpreterCheckQuery(factory); + registerInterpreterKillQueryQuery(factory); + registerInterpreterSystemQuery(factory); + registerInterpreterWatchQuery(factory); + registerInterpreterCreateUserQuery(factory); + registerInterpreterCreateRoleQuery(factory); + registerInterpreterCreateQuotaQuery(factory); + registerInterpreterCreateRowPolicyQuery(factory); + registerInterpreterCreateSettingsProfileQuery(factory); + registerInterpreterDropAccessEntityQuery(factory); + registerInterpreterMoveAccessEntityQuery(factory); + registerInterpreterDropNamedCollectionQuery(factory); + registerInterpreterGrantQuery(factory); + registerInterpreterShowCreateAccessEntityQuery(factory); + registerInterpreterShowGrantsQuery(factory); + registerInterpreterShowAccessEntitiesQuery(factory); + registerInterpreterShowAccessQuery(factory); + registerInterpreterShowPrivilegesQuery(factory); + registerInterpreterExternalDDLQuery(factory); + registerInterpreterTransactionControlQuery(factory); + registerInterpreterCreateFunctionQuery(factory); + registerInterpreterDropFunctionQuery(factory); + registerInterpreterCreateIndexQuery(factory); + registerInterpreterCreateNamedCollectionQuery(factory); + registerInterpreterDropIndexQuery(factory); + registerInterpreterBackupQuery(factory); + registerInterpreterDeleteQuery(factory); +} +} diff --git a/src/Interpreters/registerInterpreters.h b/src/Interpreters/registerInterpreters.h new file mode 100644 index 000000000000..9f0c3bbec22e --- /dev/null +++ b/src/Interpreters/registerInterpreters.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ +void registerInterpreters(); +} From 37a7ce29504eaf96546e8ba1038f6c30c1e9baaf Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 9 Jan 2024 17:17:30 +0800 Subject: [PATCH 109/114] 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 0cf6dbd80a16..0c7b99509723 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 79a1c3a968b4..68bffe9f8016 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 110/114] 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 749fce669df1..d0752a77fc71 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 000000000000..7275f9a0b2b1 --- /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 fc67bca58160..000000000000 --- 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 111/114] 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 21a1b643d987..bb12d1f251f5 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 112/114] 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 895a12313daa..e3319fe4a72a 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 @@ def process(field_name: str, field: Union[bool, str]) -> str: @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 113/114] Extend S3Helper to accept a custom s3 client and endpoint --- tests/ci/s3_helper.py | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index bc403aa50154..616d645b5a67 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -5,20 +5,19 @@ 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 compress_files import compress_file_fast from env_helper import ( - S3_TEST_REPORTS_BUCKET, - S3_BUILDS_BUCKET, - RUNNER_TEMP, CI, - S3_URL, + RUNNER_TEMP, + S3_BUILDS_BUCKET, S3_DOWNLOAD, + S3_TEST_REPORTS_BUCKET, + S3_URL, ) -from compress_files import compress_file_fast 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 @@ def upload_task(file_path: Path) -> str: 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 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 114/114] 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 c56b3c2fd0df..d356e429892b 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"