From 193b572a0581b621069684e437daf93980e3fcbc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 15:11:23 +0300 Subject: [PATCH 01/54] Add MaterializingStep. --- .../QueryPlan/MaterializingStep.cpp | 39 +++++++++++++++++++ src/Processors/QueryPlan/MaterializingStep.h | 18 +++++++++ src/Processors/ya.make | 1 + 3 files changed, 58 insertions(+) create mode 100644 src/Processors/QueryPlan/MaterializingStep.cpp create mode 100644 src/Processors/QueryPlan/MaterializingStep.h diff --git a/src/Processors/QueryPlan/MaterializingStep.cpp b/src/Processors/QueryPlan/MaterializingStep.cpp new file mode 100644 index 00000000000..f5313369020 --- /dev/null +++ b/src/Processors/QueryPlan/MaterializingStep.cpp @@ -0,0 +1,39 @@ +#include +#include +#include + +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +MaterializingStep::MaterializingStep(const DataStream & input_stream_) + : ITransformingStep(input_stream_, materializeBlock(input_stream_.header), getTraits()) +{ +} + +void MaterializingStep::transformPipeline(QueryPipeline & pipeline) +{ + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header); + }); +} + +} diff --git a/src/Processors/QueryPlan/MaterializingStep.h b/src/Processors/QueryPlan/MaterializingStep.h new file mode 100644 index 00000000000..c1ffcaeb775 --- /dev/null +++ b/src/Processors/QueryPlan/MaterializingStep.h @@ -0,0 +1,18 @@ +#pragma once +#include + +namespace DB +{ + +/// Convert one block structure to another. See ConvertingTransform. +class MaterializingStep : public ITransformingStep +{ +public: + explicit MaterializingStep(const DataStream & input_stream_); + + String getName() const override { return "Materializing"; } + + void transformPipeline(QueryPipeline & pipeline) override; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index c69d274a70e..cd8857926bb 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -105,6 +105,7 @@ SRCS( QueryPlan/ITransformingStep.cpp QueryPlan/LimitByStep.cpp QueryPlan/LimitStep.cpp + QueryPlan/MaterializingStep.cpp QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp QueryPlan/MergingSortedStep.cpp From 0bf4e8e6e912eda3f9d7131cf4581020e69c16e9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 16:22:24 +0300 Subject: [PATCH 02/54] Support reading with QueryPlan for StorageView. --- src/Processors/QueryPipeline.h | 2 + src/Processors/QueryPlan/MaterializingStep.h | 2 +- .../QueryPlan/SettingQuotaAndLimitsStep.cpp | 53 +++++++++++++++++++ .../QueryPlan/SettingQuotaAndLimitsStep.h | 42 +++++++++++++++ src/Processors/ya.make | 1 + src/Storages/StorageView.cpp | 52 ++++++++++++++++++ src/Storages/StorageView.h | 13 +++++ 7 files changed, 164 insertions(+), 1 deletion(-) create mode 100644 src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp create mode 100644 src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 45b410ab323..40aabf43ecb 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -99,6 +99,8 @@ public: void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } + void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); } + void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); }; /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); diff --git a/src/Processors/QueryPlan/MaterializingStep.h b/src/Processors/QueryPlan/MaterializingStep.h index c1ffcaeb775..72b3133dfe4 100644 --- a/src/Processors/QueryPlan/MaterializingStep.h +++ b/src/Processors/QueryPlan/MaterializingStep.h @@ -4,7 +4,7 @@ namespace DB { -/// Convert one block structure to another. See ConvertingTransform. +/// Materialize constants. See MaterializingTransform. class MaterializingStep : public ITransformingStep { public: diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp new file mode 100644 index 00000000000..73cd459fa5d --- /dev/null +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -0,0 +1,53 @@ +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +SettingQuotaAndLimitsStep::SettingQuotaAndLimitsStep( + const DataStream & input_stream_, + StoragePtr storage_, + TableLockHolder table_lock_, + StreamLocalLimits limits_, + std::shared_ptr quota_, + std::shared_ptr context_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + , storage(std::move(storage_)) + , table_lock(std::move(table_lock_)) + , limits(std::move(limits_)) + , quota(std::move(quota_)) + , context(std::move(context_)) +{ +} + +void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline) +{ + /// Table lock is stored inside pipeline here. + pipeline.addTableLock(table_lock); + + pipeline.setLimits(limits); + + if (quota) + pipeline.setQuota(quota); + + pipeline.addInterpreterContext(std::move(context)); + pipeline.addStorageHolder(std::move(storage)); +} + +} diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h new file mode 100644 index 00000000000..538d3c35b9d --- /dev/null +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -0,0 +1,42 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IStorage; +using StoragePtr = std::shared_ptr; + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +class EnabledQuota; + +/// Add limits, quota, table_lock and other stuff to pipeline. +/// Doesn't change DataStream. +class SettingQuotaAndLimitsStep : public ITransformingStep +{ +public: + SettingQuotaAndLimitsStep( + const DataStream & input_stream_, + StoragePtr storage_, + TableLockHolder table_lock_, + StreamLocalLimits limits_, + std::shared_ptr quota_, + std::shared_ptr context_); + + String getName() const override { return "SettingQuotaAndLimits"; } + + void transformPipeline(QueryPipeline & pipeline) override; + +private: + StoragePtr storage; + TableLockHolder table_lock; + StreamLocalLimits limits; + std::shared_ptr quota; + std::shared_ptr context; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index cd8857926bb..08de142479b 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -116,6 +116,7 @@ SRCS( QueryPlan/ReadFromStorageStep.cpp QueryPlan/ReadNothingStep.cpp QueryPlan/RollupStep.cpp + QueryPlan/SettingQuotaAndLimitsStep.cpp QueryPlan/TotalsHavingStep.cpp QueryPlan/UnionStep.cpp ResizeProcessor.cpp diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 4b7733c1cd2..a7cba22bebf 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -16,6 +16,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -89,6 +92,55 @@ Pipe StorageView::read( return QueryPipeline::getPipe(std::move(pipeline)); } +void StorageView::read( + QueryPlan & query_plan, + TableLockHolder table_lock, + StorageMetadataPtr metadata_snapshot, + StreamLocalLimits & limits, + std::shared_ptr quota, + const Names & column_names, + const SelectQueryInfo & query_info, + std::shared_ptr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + ASTPtr current_inner_query = metadata_snapshot->getSelectQuery().inner_query; + + if (query_info.view_query) + { + if (!query_info.view_query->as()) + throw Exception("Unexpected optimized VIEW query", ErrorCodes::LOGICAL_ERROR); + current_inner_query = query_info.view_query->clone(); + } + + InterpreterSelectWithUnionQuery interpreter(current_inner_query, *context, {}, column_names); + interpreter.buildQueryPlan(query_plan); + + /// It's expected that the columns read from storage are not constant. + /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. + auto materializing = std::make_unique(query_plan.getCurrentDataStream()); + materializing->setStepDescription("Materialize constants after VIEW subquery"); + query_plan.addStep(std::move(materializing)); + + /// And also convert to expected structure. + auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), header); + converting->setStepDescription("Convert VIEW subquery result to VIEW table structure"); + query_plan.addStep(std::move(converting)); + + /// Extend lifetime of context, table lock, storage. Set limits and quota. + auto adding_limits_and_quota = std::make_unique( + query_plan.getCurrentDataStream(), + shared_from_this(), + std::move(table_lock), + limits, + std::move(quota), + std::move(context)); + adding_limits_and_quota->setStepDescription("Set limits and quota for VIEW subquery"); + query_plan.addStep(std::move(adding_limits_and_quota)); +} + static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) { auto * select_element = select_query.tables()->children[0]->as(); diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 682c7424b98..79155209ff8 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -30,6 +30,19 @@ public: size_t max_block_size, unsigned num_streams) override; + void read( + QueryPlan & query_plan, + TableLockHolder table_lock, + StorageMetadataPtr metadata_snapshot, + StreamLocalLimits & limits, + std::shared_ptr quota, + const Names & column_names, + const SelectQueryInfo & query_info, + std::shared_ptr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot) const { replaceWithSubquery(select_query, metadata_snapshot->getSelectQuery().inner_query->clone(), view_name); From e013acb6b66e0986e2a33d5d6060ab0bd1c27dac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 16:26:34 +0300 Subject: [PATCH 03/54] Fix build. --- src/Processors/QueryPipeline.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 40aabf43ecb..03547170286 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -100,7 +100,7 @@ public: void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); } - void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); }; + void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); From c498b2b3ddb5c39afe93569aed0c9dee813f967c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 16:55:48 +0300 Subject: [PATCH 04/54] Added perftest. --- tests/performance/push_down_limit.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 0dcd9335a52..339709d552e 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,4 +1,8 @@ + CREATE VIEW numbers_view AS SELECT number from numbers_mt(100000000) order by number desc + select number from (select number from numbers(10000000) order by -number) limit 10 select number from (select number from numbers_mt(100000000) order by -number) limit 10 + + select number from numbers_view limit 100 From 50674a320fc083122c58a1023a11bc50dadfe570 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 18 Sep 2020 14:39:07 +0300 Subject: [PATCH 05/54] Refactor IStorage::read with query plan. --- src/Interpreters/InterpreterSelectQuery.cpp | 18 +++++- src/Processors/QueryPipeline.h | 1 + .../QueryPlan/ReadFromStorageStep.cpp | 55 +++-------------- .../QueryPlan/ReadFromStorageStep.h | 21 +------ .../QueryPlan/SettingQuotaAndLimitsStep.cpp | 17 +++++- .../QueryPlan/SettingQuotaAndLimitsStep.h | 4 +- src/Storages/IStorage.cpp | 11 +--- src/Storages/IStorage.h | 18 +++--- src/Storages/StorageView.cpp | 60 +++---------------- src/Storages/StorageView.h | 7 +-- 10 files changed, 66 insertions(+), 146 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cd2c16813b4..073198c3a27 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include @@ -1456,8 +1456,20 @@ void InterpreterSelectQuery::executeFetchColumns( if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) quota = context->getQuota(); - storage->read(query_plan, table_lock, metadata_snapshot, limits, leaf_limits, std::move(quota), - required_columns, query_info, context, processing_stage, max_block_size, max_streams); + storage->read(query_plan, required_columns, metadata_snapshot, + query_info, *context, processing_stage, max_block_size, max_streams); + + /// Extend lifetime of context, table lock, storage. Set limits and quota. + auto adding_limits_and_quota = std::make_unique( + query_plan.getCurrentDataStream(), + storage, + std::move(table_lock), + limits, + leaf_limits, + std::move(quota), + std::move(context)); + adding_limits_and_quota->setStepDescription("Set limits and quota after reading from storage"); + query_plan.addStep(std::move(adding_limits_and_quota)); } else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 03547170286..f923df43d76 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -100,6 +100,7 @@ public: void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); } + void setLeafLimits(const SizeLimits & limits) { pipe.setLeafLimits(limits); } void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); } /// For compatibility with IBlockInputStream. diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index b085c177ad4..a88a396193b 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -12,35 +12,19 @@ namespace DB { ReadFromStorageStep::ReadFromStorageStep( - TableLockHolder table_lock_, - StorageMetadataPtr metadata_snapshot_, - StreamLocalLimits & limits_, - SizeLimits & leaf_limits_, - std::shared_ptr quota_, - StoragePtr storage_, - const Names & required_columns_, - const SelectQueryInfo & query_info_, - std::shared_ptr context_, - QueryProcessingStage::Enum processing_stage_, - size_t max_block_size_, - size_t max_streams_) - : table_lock(std::move(table_lock_)) - , metadata_snapshot(std::move(metadata_snapshot_)) - , limits(limits_) - , leaf_limits(leaf_limits_) - , quota(std::move(quota_)) - , storage(std::move(storage_)) - , required_columns(required_columns_) - , query_info(query_info_) - , context(std::move(context_)) - , processing_stage(processing_stage_) - , max_block_size(max_block_size_) - , max_streams(max_streams_) + StoragePtr storage, + const Names & required_columns, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processing_stage, + size_t max_block_size, + size_t max_streams) { /// Note: we read from storage in constructor of step because we don't know real header before reading. /// It will be fixed when storage return QueryPlanStep itself. - Pipe pipe = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); + Pipe pipe = storage->read(required_columns, metadata_snapshot, query_info, context, processing_stage, max_block_size, max_streams); if (pipe.empty()) { @@ -83,29 +67,8 @@ ReadFromStorageStep::ReadFromStorageStep( pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); - /// Table lock is stored inside pipeline here. - pipeline->addTableLock(table_lock); - - pipe.setLimits(limits); - - /** - * Leaf size limits should be applied only for local processing of distributed queries. - * Such limits allow to control the read stage on leaf nodes and exclude the merging stage. - * Consider the case when distributed query needs to read from multiple shards. Then leaf - * limits will be applied on the shards only (including the root node) but will be ignored - * on the results merging stage. - */ - if (!storage->isRemote()) - pipe.setLeafLimits(leaf_limits); - - if (quota) - pipe.setQuota(quota); - pipeline->init(std::move(pipe)); - pipeline->addInterpreterContext(std::move(context)); - pipeline->addStorageHolder(std::move(storage)); - processors = collector.detachProcessors(); output_stream = DataStream{.header = pipeline->getHeader(), .has_single_port = pipeline->getNumStreams() == 1}; diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index 98cde63a863..59276d13081 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -23,15 +23,11 @@ class ReadFromStorageStep : public IQueryPlanStep { public: ReadFromStorageStep( - TableLockHolder table_lock, - StorageMetadataPtr metadata_snapshot, - StreamLocalLimits & limits, - SizeLimits & leaf_limits, - std::shared_ptr quota, StoragePtr storage, const Names & required_columns, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, - std::shared_ptr context, + const Context & context, QueryProcessingStage::Enum processing_stage, size_t max_block_size, size_t max_streams); @@ -45,19 +41,6 @@ public: void describePipeline(FormatSettings & settings) const override; private: - TableLockHolder table_lock; - StorageMetadataPtr metadata_snapshot; - StreamLocalLimits limits; - SizeLimits leaf_limits; - std::shared_ptr quota; - - StoragePtr storage; - const Names & required_columns; - const SelectQueryInfo & query_info; - std::shared_ptr context; - QueryProcessingStage::Enum processing_stage; - size_t max_block_size; - size_t max_streams; QueryPipelinePtr pipeline; Processors processors; diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 73cd459fa5d..2a03d1fd82f 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -24,13 +25,15 @@ SettingQuotaAndLimitsStep::SettingQuotaAndLimitsStep( const DataStream & input_stream_, StoragePtr storage_, TableLockHolder table_lock_, - StreamLocalLimits limits_, + StreamLocalLimits & limits_, + SizeLimits & leaf_limits_, std::shared_ptr quota_, std::shared_ptr context_) : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , storage(std::move(storage_)) , table_lock(std::move(table_lock_)) - , limits(std::move(limits_)) + , limits(limits_) + , leaf_limits(leaf_limits_) , quota(std::move(quota_)) , context(std::move(context_)) { @@ -43,6 +46,16 @@ void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline) pipeline.setLimits(limits); + /** + * Leaf size limits should be applied only for local processing of distributed queries. + * Such limits allow to control the read stage on leaf nodes and exclude the merging stage. + * Consider the case when distributed query needs to read from multiple shards. Then leaf + * limits will be applied on the shards only (including the root node) but will be ignored + * on the results merging stage. + */ + if (!storage->isRemote()) + pipeline.setLeafLimits(leaf_limits); + if (quota) pipeline.setQuota(quota); diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h index 538d3c35b9d..7ec4cfa91c6 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -23,7 +23,8 @@ public: const DataStream & input_stream_, StoragePtr storage_, TableLockHolder table_lock_, - StreamLocalLimits limits_, + StreamLocalLimits & limits_, + SizeLimits & leaf_limits_, std::shared_ptr quota_, std::shared_ptr context_); @@ -35,6 +36,7 @@ private: StoragePtr storage; TableLockHolder table_lock; StreamLocalLimits limits; + SizeLimits leaf_limits; std::shared_ptr quota; std::shared_ptr context; }; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 50b36ced19c..73dabca0871 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -94,21 +94,16 @@ Pipe IStorage::read( void IStorage::read( QueryPlan & query_plan, - TableLockHolder table_lock, - StorageMetadataPtr metadata_snapshot, - StreamLocalLimits & limits, - SizeLimits & leaf_limits, - std::shared_ptr quota, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, - std::shared_ptr context, + const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) { auto read_step = std::make_unique( - std::move(table_lock), std::move(metadata_snapshot), limits, leaf_limits, std::move(quota), shared_from_this(), - column_names, query_info, std::move(context), processed_stage, max_block_size, num_streams); + shared_from_this(), column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); read_step->setStepDescription("Read from " + getName()); query_plan.addStep(std::move(read_step)); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index dbd18c9558e..58a39497e38 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -285,17 +285,13 @@ public: /// Default implementation creates ReadFromStorageStep and uses usual read. virtual void read( QueryPlan & query_plan, - TableLockHolder table_lock, - StorageMetadataPtr metadata_snapshot, - StreamLocalLimits & limits, - SizeLimits & leaf_limits, - std::shared_ptr quota, - const Names & column_names, - const SelectQueryInfo & query_info, - std::shared_ptr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams); + const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/); /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index a7cba22bebf..e71228f2a23 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -55,52 +55,21 @@ Pipe StorageView::read( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const unsigned num_streams) { - Pipes pipes; - - ASTPtr current_inner_query = metadata_snapshot->getSelectQuery().inner_query; - - if (query_info.view_query) - { - if (!query_info.view_query->as()) - throw Exception("Unexpected optimized VIEW query", ErrorCodes::LOGICAL_ERROR); - current_inner_query = query_info.view_query->clone(); - } - - InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); - - auto pipeline = interpreter.execute().pipeline; - - /// It's expected that the columns read from storage are not constant. - /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - pipeline.addSimpleTransform([](const Block & header) - { - return std::make_shared(header); - }); - - /// And also convert to expected structure. - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, metadata_snapshot->getSampleBlockForColumns( - column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name); - }); - - return QueryPipeline::getPipe(std::move(pipeline)); + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); } void StorageView::read( QueryPlan & query_plan, - TableLockHolder table_lock, - StorageMetadataPtr metadata_snapshot, - StreamLocalLimits & limits, - std::shared_ptr quota, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, - std::shared_ptr context, + const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, const unsigned /*num_streams*/) @@ -114,7 +83,7 @@ void StorageView::read( current_inner_query = query_info.view_query->clone(); } - InterpreterSelectWithUnionQuery interpreter(current_inner_query, *context, {}, column_names); + InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); interpreter.buildQueryPlan(query_plan); /// It's expected that the columns read from storage are not constant. @@ -128,17 +97,6 @@ void StorageView::read( auto converting = std::make_unique(query_plan.getCurrentDataStream(), header); converting->setStepDescription("Convert VIEW subquery result to VIEW table structure"); query_plan.addStep(std::move(converting)); - - /// Extend lifetime of context, table lock, storage. Set limits and quota. - auto adding_limits_and_quota = std::make_unique( - query_plan.getCurrentDataStream(), - shared_from_this(), - std::move(table_lock), - limits, - std::move(quota), - std::move(context)); - adding_limits_and_quota->setStepDescription("Set limits and quota for VIEW subquery"); - query_plan.addStep(std::move(adding_limits_and_quota)); } static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 79155209ff8..1b43888baf3 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -32,13 +32,10 @@ public: void read( QueryPlan & query_plan, - TableLockHolder table_lock, - StorageMetadataPtr metadata_snapshot, - StreamLocalLimits & limits, - std::shared_ptr quota, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, - std::shared_ptr context, + const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; From 2ffc2b692765b6c1883b1afb6386f1875379a1b2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 18 Sep 2020 15:10:59 +0300 Subject: [PATCH 06/54] Refactor IStorage::read with query plan. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 073198c3a27..e63cf1fd78c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1467,7 +1467,7 @@ void InterpreterSelectQuery::executeFetchColumns( limits, leaf_limits, std::move(quota), - std::move(context)); + context); adding_limits_and_quota->setStepDescription("Set limits and quota after reading from storage"); query_plan.addStep(std::move(adding_limits_and_quota)); } From b26f11c00caa3af96071fdb90f0bf039598b1ff6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 18 Sep 2020 17:16:53 +0300 Subject: [PATCH 07/54] Support StorageDistributed::read for QueryPlan. --- .../ClusterProxy/IStreamFactory.h | 7 +- .../ClusterProxy/SelectStreamFactory.cpp | 54 +++++------ .../ClusterProxy/SelectStreamFactory.h | 4 +- .../ClusterProxy/executeQuery.cpp | 48 +++++++++- src/Interpreters/ClusterProxy/executeQuery.h | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 52 +++++++++++ .../QueryPlan/ReadFromPreparedSource.cpp | 4 +- .../QueryPlan/ReadFromPreparedSource.h | 16 +++- .../QueryPlan/ReadFromStorageStep.cpp | 89 ------------------- .../QueryPlan/ReadFromStorageStep.h | 49 ---------- src/Processors/QueryPlan/UnionStep.cpp | 2 +- src/Processors/QueryPlan/UnionStep.h | 2 +- src/Processors/ya.make | 1 - src/Storages/IStorage.cpp | 8 +- src/Storages/StorageDistributed.cpp | 3 +- src/Storages/StorageDistributed.h | 10 +++ 16 files changed, 166 insertions(+), 187 deletions(-) delete mode 100644 src/Processors/QueryPlan/ReadFromStorageStep.cpp delete mode 100644 src/Processors/QueryPlan/ReadFromStorageStep.h diff --git a/src/Interpreters/ClusterProxy/IStreamFactory.h b/src/Interpreters/ClusterProxy/IStreamFactory.h index 8829dc38c93..80be585d15e 100644 --- a/src/Interpreters/ClusterProxy/IStreamFactory.h +++ b/src/Interpreters/ClusterProxy/IStreamFactory.h @@ -16,6 +16,9 @@ struct SelectQueryInfo; class Pipe; using Pipes = std::vector; +class QueryPlan; +using QueryPlanPtr = std::unique_ptr; + namespace ClusterProxy { @@ -31,7 +34,9 @@ public: const String & query, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo & query_info, - Pipes & res) = 0; + std::vector & res, + Pipes & remote_pipes, + Pipes & delayed_pipes) = 0; }; } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index ed7bd2cf71f..e1b008da66b 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -14,6 +14,8 @@ #include #include #include +#include + namespace ProfileEvents { @@ -69,38 +71,25 @@ SelectStreamFactory::SelectStreamFactory( namespace { -auto createLocalPipe( - const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage) +std::unique_ptr createLocalPlan( + const ASTPtr & query_ast, + const Block & header, + const Context & context, + QueryProcessingStage::Enum processed_stage) { checkStackSize(); - InterpreterSelectQuery interpreter(query_ast, context, SelectQueryOptions(processed_stage)); auto query_plan = std::make_unique(); + InterpreterSelectQuery interpreter(query_ast, context, SelectQueryOptions(processed_stage)); interpreter.buildQueryPlan(*query_plan); - auto pipeline = std::move(*query_plan->buildQueryPipeline()); - /// Avoid going it out-of-scope for EXPLAIN - pipeline.addQueryPlan(std::move(query_plan)); - - pipeline.addSimpleTransform([&](const Block & source_header) - { - return std::make_shared( - source_header, header, ConvertingTransform::MatchColumnsMode::Name, true); - }); - - /** Materialization is needed, since from remote servers the constants come materialized. - * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, - * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. - */ - - /* Now we don't need to materialize constants, because RemoteBlockInputStream will ignore constant and take it from header. - * So, streams from different threads will always have the same header. - */ - /// return std::make_shared(stream); - - pipeline.setMaxThreads(1); - return QueryPipeline::getPipe(std::move(pipeline)); + /// Convert header structure to expected. + /// Also we ignore constants from result and replace it with constants from header. + /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. + auto converting = std::make_unique(query_plan->getCurrentDataStream(), header, true); + converting->setStepDescription("Convert block structure for query from local replica"); + query_plan->addStep(std::move(converting)); } String formattedAST(const ASTPtr & ast) @@ -119,7 +108,9 @@ void SelectStreamFactory::createForShard( const String &, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo &, - Pipes & pipes) + std::vector & plans, + Pipes & remote_pipes, + Pipes & delayed_pipes) { bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -136,7 +127,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - pipes.emplace_back(createLocalPipe(modified_query_ast, header, context, processed_stage)); + plans.emplace_back(createLocalPlan(modified_query_ast, header, context, processed_stage)); }; String modified_query = formattedAST(modified_query_ast); @@ -149,7 +140,7 @@ void SelectStreamFactory::createForShard( if (!table_func_ptr) remote_query_executor->setMainTable(main_table); - pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes)); + remote_pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes)); }; const auto & settings = context.getSettingsRef(); @@ -276,7 +267,10 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return createLocalPipe(modified_query_ast, header, context, stage); + { + auto plan = createLocalPlan(modified_query_ast, header, context, stage); + return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline())); + } else { std::vector connections; @@ -291,7 +285,7 @@ void SelectStreamFactory::createForShard( } }; - pipes.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes)); + delayed_pipes.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes)); } else emplace_remote_stream(); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 80f72fd0024..c3dfc150d6d 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -39,7 +39,9 @@ public: const String & query, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo & query_info, - Pipes & pipes) override; + std::vector & res, + Pipes & remote_pipes, + Pipes & delayed_pipes) override; private: const Block header; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 1ebd3009ff7..4618e6122be 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -7,6 +7,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -74,13 +77,16 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin return new_context; } -Pipe executeQuery( +void executeQuery( + QueryPlan & query_plan, IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info) { assert(log); - Pipes res; + std::vector plans; + Pipes remote_pipes; + Pipes delayed_pipes; const std::string query = queryToString(query_ast); @@ -104,9 +110,43 @@ Pipe executeQuery( throttler = user_level_throttler; for (const auto & shard_info : cluster->getShardsInfo()) - stream_factory.createForShard(shard_info, query, query_ast, new_context, throttler, query_info, res); + stream_factory.createForShard(shard_info, query, query_ast, new_context, throttler, query_info, plans, remote_pipes, delayed_pipes); - return Pipe::unitePipes(std::move(res)); + if (!remote_pipes.empty()) + { + auto plan = std::make_unique(); + auto read_from_remote = std::make_unique(Pipe::unitePipes(std::move(remote_pipes))); + read_from_remote->setStepDescription("Read from remote replica"); + plan->addStep(std::move(read_from_remote)); + plans.emplace_back(std::move(plan)); + } + + if (!delayed_pipes.empty()) + { + auto plan = std::make_unique(); + auto read_from_remote = std::make_unique(Pipe::unitePipes(std::move(delayed_pipes))); + read_from_remote->setStepDescription("Read from delayed local replica"); + plan->addStep(std::move(read_from_remote)); + plans.emplace_back(std::move(plan)); + } + + if (plans.empty()) + return; + + if (plans.size() == 1) + { + query_plan = std::move(*plans.front()); + return; + } + + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto header = input_streams.front().header; + auto union_step = std::make_unique(std::move(input_streams), header); + query_plan.unitePlans(std::move(union_step), std::move(plans)); } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index f0d9539770d..1adc689c905 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -12,6 +12,7 @@ class Cluster; struct SelectQueryInfo; class Pipe; +class QueryPlan; namespace ClusterProxy { @@ -25,7 +26,8 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin /// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read. /// `stream_factory` object encapsulates the logic of creating streams for a different type of query /// (currently SELECT, DESCRIBE). -Pipe executeQuery( +void executeQuery( + QueryPlan & query_plan, IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e63cf1fd78c..30da2e7d6f2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -31,9 +31,11 @@ #include #include +#include #include #include #include +#include #include #include #include @@ -1102,6 +1104,48 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se return limits; } +static void addEmptySource(QueryPlan & query_plan, const Block & header, SelectQueryInfo & query_info) +{ + Pipe pipe(std::make_shared(header)); + + if (query_info.prewhere_info) + { + if (query_info.prewhere_info->alias_actions) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, query_info.prewhere_info->alias_actions); + }); + } + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + query_info.prewhere_info->prewhere_actions, + query_info.prewhere_info->prewhere_column_name, + query_info.prewhere_info->remove_prewhere_column); + }); + + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table + if (query_info.prewhere_info->remove_columns_actions) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, query_info.prewhere_info->remove_columns_actions); + }); + } + } + + 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)); +} + void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) @@ -1459,6 +1503,14 @@ void InterpreterSelectQuery::executeFetchColumns( storage->read(query_plan, required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto header = metadata_snapshot->getSampleBlockForColumns( + required_columns, storage->getVirtuals(), storage->getStorageID()); + addEmptySource(query_plan, header, query_info); + } + /// Extend lifetime of context, table lock, storage. Set limits and quota. auto adding_limits_and_quota = std::make_unique( query_plan.getCurrentDataStream(), diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 6f0d1693ce0..dcfe609e070 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -14,7 +14,9 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_); + explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_ = nullptr); - String getName() const override { return "ReadNothing"; } + String getName() const override { return "ReadFromPreparedSource"; } void initializePipeline(QueryPipeline & pipeline) override; @@ -20,4 +20,16 @@ private: std::shared_ptr context; }; +class ReadFromStorageStep : public ReadFromPreparedSource +{ +public: + ReadFromStorageStep(Pipe pipe, String storage_name) + : ReadFromPreparedSource(std::move(pipe)) + { + setStepDescription(storage_name); + } + + String getName() const override { return "ReadFromStorage"; } +}; + } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp deleted file mode 100644 index a88a396193b..00000000000 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ /dev/null @@ -1,89 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -ReadFromStorageStep::ReadFromStorageStep( - StoragePtr storage, - const Names & required_columns, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processing_stage, - size_t max_block_size, - size_t max_streams) -{ - /// Note: we read from storage in constructor of step because we don't know real header before reading. - /// It will be fixed when storage return QueryPlanStep itself. - - Pipe pipe = storage->read(required_columns, metadata_snapshot, query_info, context, processing_stage, max_block_size, max_streams); - - if (pipe.empty()) - { - pipe = Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); - - if (query_info.prewhere_info) - { - if (query_info.prewhere_info->alias_actions) - { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, query_info.prewhere_info->alias_actions); - }); - } - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - query_info.prewhere_info->prewhere_actions, - query_info.prewhere_info->prewhere_column_name, - query_info.prewhere_info->remove_prewhere_column); - }); - - // To remove additional columns - // In some cases, we did not read any marks so that the pipeline.streams is empty - // Thus, some columns in prewhere are not removed as expected - // This leads to mismatched header in distributed table - if (query_info.prewhere_info->remove_columns_actions) - { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, query_info.prewhere_info->remove_columns_actions); - }); - } - } - } - - pipeline = std::make_unique(); - QueryPipelineProcessorsCollector collector(*pipeline, this); - - pipeline->init(std::move(pipe)); - - processors = collector.detachProcessors(); - - output_stream = DataStream{.header = pipeline->getHeader(), .has_single_port = pipeline->getNumStreams() == 1}; -} - -ReadFromStorageStep::~ReadFromStorageStep() = default; - -QueryPipelinePtr ReadFromStorageStep::updatePipeline(QueryPipelines) -{ - return std::move(pipeline); -} - -void ReadFromStorageStep::describePipeline(FormatSettings & settings) const -{ - IQueryPlanStep::describePipeline(processors, settings); -} - -} diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h deleted file mode 100644 index 59276d13081..00000000000 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ /dev/null @@ -1,49 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ - -class IStorage; -using StoragePtr = std::shared_ptr; - -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; - -struct SelectQueryInfo; - -struct PrewhereInfo; - -class EnabledQuota; - -/// Reads from storage. -class ReadFromStorageStep : public IQueryPlanStep -{ -public: - ReadFromStorageStep( - StoragePtr storage, - const Names & required_columns, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processing_stage, - size_t max_block_size, - size_t max_streams); - - ~ReadFromStorageStep() override; - - String getName() const override { return "ReadFromStorage"; } - - QueryPipelinePtr updatePipeline(QueryPipelines) override; - - void describePipeline(FormatSettings & settings) const override; - -private: - - QueryPipelinePtr pipeline; - Processors processors; -}; - -} diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index 1e74046b071..630ff53f47d 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -30,7 +30,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) return pipeline; } - *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header ,max_threads); + *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header, max_threads); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/UnionStep.h b/src/Processors/QueryPlan/UnionStep.h index 9e00e24279b..e2e1f2c9efa 100644 --- a/src/Processors/QueryPlan/UnionStep.h +++ b/src/Processors/QueryPlan/UnionStep.h @@ -9,7 +9,7 @@ class UnionStep : public IQueryPlanStep { public: /// max_threads is used to limit the number of threads for result pipeline. - UnionStep(DataStreams input_streams_, Block result_header, size_t max_threads_); + UnionStep(DataStreams input_streams_, Block result_header, size_t max_threads_ = 0); String getName() const override { return "Union"; } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 08de142479b..cd18ea3deb0 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -113,7 +113,6 @@ SRCS( QueryPlan/PartialSortingStep.cpp QueryPlan/QueryPlan.cpp QueryPlan/ReadFromPreparedSource.cpp - QueryPlan/ReadFromStorageStep.cpp QueryPlan/ReadNothingStep.cpp QueryPlan/RollupStep.cpp QueryPlan/SettingQuotaAndLimitsStep.cpp diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 73dabca0871..fab3018911f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -102,10 +102,8 @@ void IStorage::read( size_t max_block_size, unsigned num_streams) { - auto read_step = std::make_unique( - shared_from_this(), column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - - read_step->setStepDescription("Read from " + getName()); + auto pipe = read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + auto read_step = std::make_unique(std::move(pipe), getName()); query_plan.addStep(std::move(read_step)); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 596681002ab..268d1b9287f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -494,6 +494,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con } Pipe StorageDistributed::read( + QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -537,7 +538,7 @@ Pipe StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, StorageID{remote_database, remote_table}, scalars, has_virtual_shard_num_column, context.getExternalTables()); - return ClusterProxy::executeQuery(select_stream_factory, cluster, log, + ClusterProxy::executeQuery(query_plan, select_stream_factory, cluster, log, modified_query_ast, context, context.getSettingsRef(), query_info); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 7e4e9f56ab4..c4e213d2276 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -77,6 +77,16 @@ public: size_t max_block_size, unsigned num_streams) override; + Pipe StorageDistributed::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) + bool supportsParallelInsert() const override { return true; } BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; From 576ffadb170747665dc3a7b9d1b6918e58358096 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 25 Sep 2020 16:19:26 +0300 Subject: [PATCH 08/54] Fix explain for ISourceStep. --- .../ClusterProxy/SelectStreamFactory.cpp | 2 ++ .../ClusterProxy/SelectStreamFactory.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- src/Interpreters/InterpreterSelectQuery.h | 2 ++ src/Processors/QueryPlan/ISourceStep.cpp | 3 ++- src/Processors/QueryPlan/ISourceStep.h | 2 +- .../QueryPlan/ReadFromPreparedSource.cpp | 5 ++++- src/Processors/QueryPlan/ReadFromPreparedSource.h | 4 ++-- src/Storages/IStorage.cpp | 13 +++++++++++-- src/Storages/StorageDistributed.cpp | 14 ++++++++++++++ src/Storages/StorageDistributed.h | 2 +- 11 files changed, 43 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index e1b008da66b..2e6587a32f6 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -90,6 +90,8 @@ std::unique_ptr createLocalPlan( auto converting = std::make_unique(query_plan->getCurrentDataStream(), header, true); converting->setStepDescription("Convert block structure for query from local replica"); query_plan->addStep(std::move(converting)); + + return query_plan; } String formattedAST(const ASTPtr & ast) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index c3dfc150d6d..9b57b92ed50 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -39,7 +39,7 @@ public: const String & query, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo & query_info, - std::vector & res, + std::vector & plans, Pipes & remote_pipes, Pipes & delayed_pipes) override; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 228ccfae39e..5a6bb5770cf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1107,9 +1107,9 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se return limits; } -static void addEmptySource(QueryPlan & query_plan, const Block & header, SelectQueryInfo & query_info) +void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info) { - Pipe pipe(std::make_shared(header)); + Pipe pipe(std::make_shared(source_header)); if (query_info.prewhere_info) { @@ -1511,7 +1511,7 @@ void InterpreterSelectQuery::executeFetchColumns( { auto header = metadata_snapshot->getSampleBlockForColumns( required_columns, storage->getVirtuals(), storage->getStorageID()); - addEmptySource(query_plan, header, query_info); + addEmptySourceToQueryPlan(query_plan, header, query_info); } /// Extend lifetime of context, table lock, storage. Set limits and quota. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 455b1a1e623..5ed938af9f3 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -94,6 +94,8 @@ public: const SelectQueryInfo & getQueryInfo() const { return query_info; } + static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info); + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index cf68104f18c..1796f033896 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -14,7 +14,8 @@ QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines) auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); initializePipeline(*pipeline); - processors = collector.detachProcessors(); + auto added_processors = collector.detachProcessors(); + processors.insert(processors.end(), added_processors.begin(), added_processors.end()); return pipeline; } diff --git a/src/Processors/QueryPlan/ISourceStep.h b/src/Processors/QueryPlan/ISourceStep.h index 54bc19957f4..fdb3dd566cb 100644 --- a/src/Processors/QueryPlan/ISourceStep.h +++ b/src/Processors/QueryPlan/ISourceStep.h @@ -16,7 +16,7 @@ public: void describePipeline(FormatSettings & settings) const override; -private: +protected: /// We collect processors got after pipeline transformation. Processors processors; }; diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index dcfe609e070..2d4b759b637 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -5,7 +5,7 @@ namespace DB { ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_) - : ISourceStep(DataStream{.header = pipe_.getHeader(), .has_single_port = true}) + : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) , context(std::move(context_)) { @@ -13,6 +13,9 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr #include #include +#include namespace DB @@ -103,8 +104,16 @@ void IStorage::read( unsigned num_streams) { auto pipe = read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - auto read_step = std::make_unique(std::move(pipe), getName()); - query_plan.addStep(std::move(read_step)); + if (pipe.empty()) + { + auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); + } + else + { + auto read_step = std::make_unique(std::move(pipe), getName()); + query_plan.addStep(std::move(read_step)); + } } Pipe IStorage::alterPartition( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b27b7794932..6e7d7ea41bd 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -495,6 +495,20 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con } Pipe StorageDistributed::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const unsigned num_streams) +{ + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); +} + +void StorageDistributed::read( QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 3d02d196c7e..df0ca05e1c6 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -77,7 +77,7 @@ public: size_t max_block_size, unsigned num_streams) override; - Pipe StorageDistributed::read( + void read( QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, From 5ac6bc071dc5066cd4d9759769b03898e6a3ec06 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Sep 2020 19:21:58 +0300 Subject: [PATCH 09/54] QueryPlan for StorageBuffer and StorageMaterializedView read. --- src/Processors/QueryPlan/AddingMissedStep.cpp | 45 +++++ src/Processors/QueryPlan/AddingMissedStep.h | 28 ++++ .../QueryPlan/SettingQuotaAndLimitsStep.cpp | 7 +- src/Processors/ya.make | 1 + src/Storages/StorageBuffer.cpp | 158 ++++++++++++------ src/Storages/StorageBuffer.h | 10 ++ src/Storages/StorageMaterializedView.cpp | 20 ++- src/Storages/StorageMaterializedView.h | 10 ++ ...84_shard_distributed_group_by_no_merge.sql | 4 +- 9 files changed, 226 insertions(+), 57 deletions(-) create mode 100644 src/Processors/QueryPlan/AddingMissedStep.cpp create mode 100644 src/Processors/QueryPlan/AddingMissedStep.h diff --git a/src/Processors/QueryPlan/AddingMissedStep.cpp b/src/Processors/QueryPlan/AddingMissedStep.cpp new file mode 100644 index 00000000000..f2d06c033b1 --- /dev/null +++ b/src/Processors/QueryPlan/AddingMissedStep.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +AddingMissedStep::AddingMissedStep( + const DataStream & input_stream_, + Block result_header_, + const ColumnDefaults & column_defaults_, + const Context & context_) + : ITransformingStep(input_stream_, result_header_, getTraits()) + , column_defaults(column_defaults_) + , context(context_) +{ + updateDistinctColumns(output_stream->header, output_stream->distinct_columns); +} + +void AddingMissedStep::transformPipeline(QueryPipeline & pipeline) +{ + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, output_stream->header, column_defaults, context); + }); +} + +} diff --git a/src/Processors/QueryPlan/AddingMissedStep.h b/src/Processors/QueryPlan/AddingMissedStep.h new file mode 100644 index 00000000000..77075a410a5 --- /dev/null +++ b/src/Processors/QueryPlan/AddingMissedStep.h @@ -0,0 +1,28 @@ +#pragma once +#include + +namespace DB +{ + +struct ColumnDefault; +using ColumnDefaults = std::unordered_map; + +/// Convert one block structure to another. See ConvertingTransform. +class AddingMissedStep : public ITransformingStep +{ +public: + AddingMissedStep(const DataStream & input_stream_, + Block result_header_, + const ColumnDefaults & column_defaults_, + const Context & context_); + + String getName() const override { return "AddingMissed"; } + + void transformPipeline(QueryPipeline & pipeline) override; + +private: + const ColumnDefaults column_defaults; + const Context & context; +}; + +} diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 2a03d1fd82f..5b05ad77d6c 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -59,8 +59,11 @@ void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline) if (quota) pipeline.setQuota(quota); - pipeline.addInterpreterContext(std::move(context)); - pipeline.addStorageHolder(std::move(storage)); + if (context) + pipeline.addInterpreterContext(std::move(context)); + + if (storage) + pipeline.addStorageHolder(std::move(storage)); } } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index cd18ea3deb0..b5afc1ada3a 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -89,6 +89,7 @@ SRCS( printPipeline.cpp QueryPipeline.cpp QueryPlan/AddingDelayedSourceStep.cpp + QueryPlan/AddingMissedStep.cpp QueryPlan/AggregatingStep.cpp QueryPlan/ArrayJoinStep.cpp QueryPlan/ConvertingStep.cpp diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 14f188275e5..162463a4e33 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -22,10 +22,13 @@ #include #include #include -#include +#include #include #include #include +#include +#include +#include namespace ProfileEvents @@ -147,6 +150,21 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context Pipe StorageBuffer::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const unsigned num_streams) +{ + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); +} + +void StorageBuffer::read( + QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -155,8 +173,6 @@ Pipe StorageBuffer::read( size_t max_block_size, unsigned num_streams) { - Pipe pipe_from_dst; - if (destination_id) { auto destination = DatabaseCatalog::instance().getTable(destination_id, context); @@ -182,8 +198,8 @@ Pipe StorageBuffer::read( query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, destination_metadata_snapshot); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. - pipe_from_dst = destination->read( - column_names, destination_metadata_snapshot, query_info, + destination->read( + query_plan, column_names, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } else @@ -217,25 +233,45 @@ Pipe StorageBuffer::read( } else { - pipe_from_dst = destination->read( - columns_intersection, destination_metadata_snapshot, query_info, - context, processed_stage, max_block_size, num_streams); + destination->read( + query_plan, columns_intersection, destination_metadata_snapshot, query_info, + context, processed_stage, max_block_size, num_streams); - pipe_from_dst.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, header_after_adding_defaults, + auto adding_missed = std::make_unique( + query_plan.getCurrentDataStream(), + header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context); - }); - pipe_from_dst.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared( - stream_header, header, ConvertingTransform::MatchColumnsMode::Name); - }); + adding_missed->setStepDescription("Add columns missing in destination table"); + query_plan.addStep(std::move(adding_missed)); + + auto converting = std::make_unique( + query_plan.getCurrentDataStream(), + header); + + converting->setStepDescription("Convert destination table columns to Buffer table structure"); + query_plan.addStep(std::move(converting)); } } - pipe_from_dst.addTableLock(destination_lock); + if (query_plan.isInitialized()) + { + StreamLocalLimits limits; + SizeLimits leaf_limits; + + /// Add table lock for destination table. + auto adding_limits_and_quota = std::make_unique( + query_plan.getCurrentDataStream(), + destination, + std::move(destination_lock), + limits, + leaf_limits, + nullptr, + nullptr); + + adding_limits_and_quota->setStepDescription("Lock destination table for Buffer"); + query_plan.addStep(std::move(adding_limits_and_quota)); + } } Pipe pipe_from_buffers; @@ -248,49 +284,73 @@ Pipe StorageBuffer::read( pipe_from_buffers = Pipe::unitePipes(std::move(pipes_from_buffers)); } - /// Convert pipes from table to structure from buffer. - if (!pipe_from_buffers.empty() && !pipe_from_dst.empty() - && !blocksHaveEqualStructure(pipe_from_buffers.getHeader(), pipe_from_dst.getHeader())) - { - pipe_from_dst.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - pipe_from_buffers.getHeader(), - ConvertingTransform::MatchColumnsMode::Name); - }); - } + if (pipe_from_buffers.empty()) + return; + + QueryPlan buffers_plan; /** If the sources from the table were processed before some non-initial stage of query execution, * then sources from the buffers must also be wrapped in the processing pipeline before the same stage. */ if (processed_stage > QueryProcessingStage::FetchColumns) - pipe_from_buffers = QueryPipeline::getPipe( - InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), - SelectQueryOptions(processed_stage)).execute().pipeline); - - if (query_info.prewhere_info) { - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, query_info.prewhere_info->prewhere_actions, - query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); - }); - - if (query_info.prewhere_info->alias_actions) + auto interpreter = InterpreterSelectQuery( + query_info.query, context, std::move(pipe_from_buffers), + SelectQueryOptions(processed_stage)); + interpreter.buildQueryPlan(buffers_plan); + } + else + { + if (query_info.prewhere_info) { pipe_from_buffers.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions); + return std::make_shared( + header, query_info.prewhere_info->prewhere_actions, + query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); + + if (query_info.prewhere_info->alias_actions) + { + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, query_info.prewhere_info->alias_actions); + }); + } } + + auto read_from_buffers = std::make_unique(std::move(pipe_from_buffers)); + read_from_buffers->setStepDescription("Read from buffers of Buffer table"); + buffers_plan.addStep(std::move(read_from_buffers)); } - Pipes pipes; - pipes.emplace_back(std::move(pipe_from_dst)); - pipes.emplace_back(std::move(pipe_from_buffers)); - return Pipe::unitePipes(std::move(pipes)); + if (!query_plan.isInitialized()) + { + query_plan = std::move(buffers_plan); + return; + } + + auto result_header = buffers_plan.getCurrentDataStream().header; + + /// Convert structure from table to structure from buffer. + if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) + { + auto converting = std::make_unique(query_plan.getCurrentDataStream(), result_header); + query_plan.addStep(std::move(converting)); + } + + DataStreams input_streams; + input_streams.emplace_back(query_plan.getCurrentDataStream()); + input_streams.emplace_back(buffers_plan.getCurrentDataStream()); + + std::vector> plans; + plans.emplace_back(std::make_unique(std::move(query_plan))); + plans.emplace_back(std::make_unique(std::move(buffers_plan))); + query_plan = QueryPlan(); + + auto union_step = std::make_unique(std::move(input_streams), result_header); + union_step->setStepDescription("Unite sources from Buffer table"); + query_plan.unitePlans(std::move(union_step), std::move(plans)); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index b18b574ec6c..406e6a51fdb 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -65,6 +65,16 @@ public: size_t max_block_size, unsigned num_streams) override; + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + bool supportsParallelInsert() const override { return true; } BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a2e3fae0951..69669c0b680 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -107,6 +107,21 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(cons } Pipe StorageMaterializedView::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const unsigned num_streams) +{ + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); +} + +void StorageMaterializedView::read( + QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, @@ -122,10 +137,7 @@ Pipe StorageMaterializedView::read( if (query_info.order_optimizer) query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); - Pipe pipe = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - pipe.addTableLock(lock); - - return pipe; + storage->read(query_plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 1ee4246c7f1..b7e60649601 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -80,6 +80,16 @@ public: size_t max_block_size, unsigned num_streams) override; + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + Strings getDataPaths() const override; private: diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql index e7174c5b56b..cb572cc542f 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql @@ -10,9 +10,9 @@ SET max_threads=1; SET optimize_move_functions_out_of_any=0; SELECT 'LIMIT'; -SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1 SETTINGS distributed_group_by_no_merge=2; +SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1 ) ORDER BY shard_num SETTINGS distributed_group_by_no_merge=2; SELECT 'OFFSET'; -SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1, 1 SETTINGS distributed_group_by_no_merge=2; +SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1, 1) ORDER BY shard_num SETTINGS distributed_group_by_no_merge=2; SELECT 'ALIAS'; SELECT dummy AS d FROM remote('127.0.0.{2,3}', system.one) ORDER BY d SETTINGS distributed_group_by_no_merge=2; From fcd40d041fc011e24650c199d1f1f93149db2b61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 15:54:33 +0300 Subject: [PATCH 10/54] Update test. --- tests/queries/0_stateless/01508_explain_header.reference | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01508_explain_header.reference b/tests/queries/0_stateless/01508_explain_header.reference index 50216432e14..a1f0b44c224 100644 --- a/tests/queries/0_stateless/01508_explain_header.reference +++ b/tests/queries/0_stateless/01508_explain_header.reference @@ -3,5 +3,7 @@ Header: x UInt8 Expression (Before ORDER BY and SELECT) Header: _dummy UInt8 1 UInt8 - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) Header: dummy UInt8 + ReadFromStorage (SystemOne) + Header: dummy UInt8 From 49773d6761570ecd97544ed4b980af8914f52828 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 18:53:12 +0300 Subject: [PATCH 11/54] Update test. --- .../0_stateless/00184_shard_distributed_group_by_no_merge.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql index cb572cc542f..9912e083777 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql @@ -10,9 +10,9 @@ SET max_threads=1; SET optimize_move_functions_out_of_any=0; SELECT 'LIMIT'; -SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1 ) ORDER BY shard_num SETTINGS distributed_group_by_no_merge=2; +SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one)) ORDER BY shard_num LIMIT 1 SETTINGS distributed_group_by_no_merge=2; SELECT 'OFFSET'; -SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1, 1) ORDER BY shard_num SETTINGS distributed_group_by_no_merge=2; +SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one)) ORDER BY shard_num LIMIT 1, 1 SETTINGS distributed_group_by_no_merge=2; SELECT 'ALIAS'; SELECT dummy AS d FROM remote('127.0.0.{2,3}', system.one) ORDER BY d SETTINGS distributed_group_by_no_merge=2; From cb221528787f04b861b024452231c3649accbc83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 18:54:16 +0300 Subject: [PATCH 12/54] Fix build. --- src/Processors/QueryPlan/AddingMissedStep.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/AddingMissedStep.h b/src/Processors/QueryPlan/AddingMissedStep.h index 77075a410a5..8e3b8148b6a 100644 --- a/src/Processors/QueryPlan/AddingMissedStep.h +++ b/src/Processors/QueryPlan/AddingMissedStep.h @@ -1,12 +1,10 @@ #pragma once #include +#include namespace DB { -struct ColumnDefault; -using ColumnDefaults = std::unordered_map; - /// Convert one block structure to another. See ConvertingTransform. class AddingMissedStep : public ITransformingStep { @@ -21,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; private: - const ColumnDefaults column_defaults; + ColumnDefaults column_defaults; const Context & context; }; From ec64def384bf23288450e6a7a4b005d748a46245 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Oct 2020 20:34:22 +0300 Subject: [PATCH 13/54] Use QueryPlan while reading from MergeTree. --- src/Processors/QueryPipeline.cpp | 6 + src/Processors/QueryPipeline.h | 5 + .../QueryPlan/AddingConstColumnStep.cpp | 41 +++ .../QueryPlan/AddingConstColumnStep.h | 22 ++ src/Processors/QueryPlan/MergingFinal.cpp | 159 ++++++++++ src/Processors/QueryPlan/MergingFinal.h | 35 +++ src/Processors/QueryPlan/QueryPlan.h | 5 +- src/Processors/QueryPlan/ReverseRowsStep.cpp | 37 +++ src/Processors/QueryPlan/ReverseRowsStep.h | 18 ++ .../Transforms/AddingConstColumnTransform.h | 33 +- src/Processors/ya.make | 3 + src/Storages/IStorage.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 284 ++++++++---------- .../MergeTree/MergeTreeDataSelectExecutor.h | 10 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 9 +- src/Storages/StorageMerge.cpp | 9 +- src/Storages/StorageMergeTree.cpp | 20 +- src/Storages/StorageMergeTree.h | 10 + src/Storages/StorageReplicatedMergeTree.cpp | 23 +- src/Storages/StorageReplicatedMergeTree.h | 10 + 20 files changed, 546 insertions(+), 194 deletions(-) create mode 100644 src/Processors/QueryPlan/AddingConstColumnStep.cpp create mode 100644 src/Processors/QueryPlan/AddingConstColumnStep.h create mode 100644 src/Processors/QueryPlan/MergingFinal.cpp create mode 100644 src/Processors/QueryPlan/MergingFinal.h create mode 100644 src/Processors/QueryPlan/ReverseRowsStep.cpp create mode 100644 src/Processors/QueryPlan/ReverseRowsStep.h diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 4cbb4d9edb7..8280923b1e3 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -96,6 +96,12 @@ void QueryPipeline::addTransform(ProcessorPtr transform) pipe.addTransform(std::move(transform)); } +void QueryPipeline::transform(const Transformer & transformer) +{ + checkInitializedAndNotCompleted(); + pipe.transform(transformer); +} + void QueryPipeline::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index ca6ec76bb0b..d91cfe89840 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -53,6 +53,11 @@ public: void addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter); /// Add transform with getNumStreams() input ports. void addTransform(ProcessorPtr transform); + + using Transformer = std::function; + /// Transform pipeline in general way. + void transform(const Transformer & transformer); + /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. diff --git a/src/Processors/QueryPlan/AddingConstColumnStep.cpp b/src/Processors/QueryPlan/AddingConstColumnStep.cpp new file mode 100644 index 00000000000..27c7720e58e --- /dev/null +++ b/src/Processors/QueryPlan/AddingConstColumnStep.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +AddingConstColumnStep::AddingConstColumnStep(const DataStream & input_stream_, ColumnWithTypeAndName column_) + : ITransformingStep(input_stream_, + AddingConstColumnTransform::transformHeader(input_stream_.header, column_), + getTraits()) + , column(std::move(column_)) +{ +} + +void AddingConstColumnStep::transformPipeline(QueryPipeline & pipeline) +{ + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, column); + }); +} + +} diff --git a/src/Processors/QueryPlan/AddingConstColumnStep.h b/src/Processors/QueryPlan/AddingConstColumnStep.h new file mode 100644 index 00000000000..baa63873f21 --- /dev/null +++ b/src/Processors/QueryPlan/AddingConstColumnStep.h @@ -0,0 +1,22 @@ +#pragma once +#include + +namespace DB +{ + +/// Adds a materialized const column with a specified value. +class AddingConstColumnStep : public ITransformingStep +{ +public: + AddingConstColumnStep(const DataStream & input_stream_, ColumnWithTypeAndName column_); + + String getName() const override { return "AddingConstColumn"; } + + void transformPipeline(QueryPipeline & pipeline) override; + +private: + ColumnWithTypeAndName column; +}; + +} + diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp new file mode 100644 index 00000000000..cd9d6abb4c3 --- /dev/null +++ b/src/Processors/QueryPlan/MergingFinal.cpp @@ -0,0 +1,159 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +MergingFinal::MergingFinal( + const DataStream & input_stream, + size_t num_output_streams_, + SortDescription sort_description_, + MergeTreeData::MergingParams params_, + Names partition_key_columns_, + size_t max_block_size_) + : ITransformingStep(input_stream, input_stream.header, getTraits()) + , num_output_streams(num_output_streams_) + , sort_description(std::move(sort_description_)) + , merging_params(std::move(params_)) + , partition_key_columns(std::move(partition_key_columns_)) + , max_block_size(max_block_size_) +{ + /// TODO: check input_stream is partially sorted (each port) by the same description. +// output_stream->sort_description = sort_description; +// output_stream->sort_mode = DataStream::SortMode::Stream; +} + +void MergingFinal::transformPipeline(QueryPipeline & pipeline) +{ + const auto & header = pipeline.getHeader(); + size_t num_outputs = pipeline.getNumStreams(); + + auto get_merging_processor = [&]() -> MergingTransformPtr + { + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + { + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + } + + case MergeTreeData::MergingParams::Collapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, true, max_block_size); + + case MergeTreeData::MergingParams::Summing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); + + case MergeTreeData::MergingParams::Aggregating: + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + + case MergeTreeData::MergingParams::Replacing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.version_column, max_block_size); + + case MergeTreeData::MergingParams::VersionedCollapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, max_block_size); + + case MergeTreeData::MergingParams::Graphite: + throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); + } + + __builtin_unreachable(); + }; + + if (num_outputs <= 1 || sort_description.empty()) + { + pipeline.addTransform(get_merging_processor()); + return; + } + + ColumnNumbers key_columns; + key_columns.reserve(sort_description.size()); + + for (auto & desc : sort_description) + { + if (!desc.column_name.empty()) + key_columns.push_back(header.getPositionByName(desc.column_name)); + else + key_columns.emplace_back(desc.column_number); + } + + pipeline.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, num_outputs, key_columns); + }); + + pipeline.transform([&](OutputPortRawPtrs ports) + { + Processors processors; + std::vector output_ports; + processors.reserve(ports.size() + num_outputs); + output_ports.reserve(ports.size()); + + for (auto & port : ports) + { + auto copier = std::make_shared(header, num_outputs); + connect(*port, copier->getInputPort()); + output_ports.emplace_back(copier->getOutputs().begin()); + processors.emplace_back(std::move(copier)); + } + + for (size_t i = 0; i < num_outputs; ++i) + { + auto merge = get_merging_processor(); + merge->setSelectorPosition(i); + auto input = merge->getInputs().begin(); + + /// Connect i-th merge with i-th input port of every copier. + for (size_t j = 0; j < ports.size(); ++j) + { + connect(*output_ports[j], *input); + ++output_ports[j]; + ++input; + } + + processors.emplace_back(std::move(merge)); + } + + return processors; + }); +} + +void MergingFinal::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + settings.out << prefix << "Sort description: "; + dumpSortDescription(sort_description, input_streams.front().header, settings.out); + settings.out << '\n'; +} + +} diff --git a/src/Processors/QueryPlan/MergingFinal.h b/src/Processors/QueryPlan/MergingFinal.h new file mode 100644 index 00000000000..c01f5c7f9a1 --- /dev/null +++ b/src/Processors/QueryPlan/MergingFinal.h @@ -0,0 +1,35 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +/// Merge streams of data into single sorted stream. +class MergingFinal : public ITransformingStep +{ +public: + explicit MergingFinal( + const DataStream & input_stream, + size_t num_output_streams_, + SortDescription sort_description_, + MergeTreeData::MergingParams params_, + Names partition_key_columns_, + size_t max_block_size_); + + String getName() const override { return "MergingFinal"; } + + void transformPipeline(QueryPipeline & pipeline) override; + + void describeActions(FormatSettings & settings) const override; + +private: + size_t num_output_streams; + SortDescription sort_description; + MergeTreeData::MergingParams merging_params; + Names partition_key_columns; + size_t max_block_size; +}; + +} diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 6296eac7502..4f558e04c55 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -17,6 +17,9 @@ using QueryPipelinePtr = std::unique_ptr; class Context; class WriteBuffer; +class QueryPlan; +using QueryPlanPtr = std::unique_ptr; + /// A tree of query steps. /// The goal of QueryPlan is to build QueryPipeline. /// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimisations. @@ -28,7 +31,7 @@ public: QueryPlan(QueryPlan &&); QueryPlan & operator=(QueryPlan &&); - void unitePlans(QueryPlanStepPtr step, std::vector> plans); + void unitePlans(QueryPlanStepPtr step, std::vector plans); void addStep(QueryPlanStepPtr step); bool isInitialized() const { return root != nullptr; } /// Tree is not empty diff --git a/src/Processors/QueryPlan/ReverseRowsStep.cpp b/src/Processors/QueryPlan/ReverseRowsStep.cpp new file mode 100644 index 00000000000..32e16937611 --- /dev/null +++ b/src/Processors/QueryPlan/ReverseRowsStep.cpp @@ -0,0 +1,37 @@ +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = false, + .preserves_number_of_streams = true, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = true, + } + }; +} + +ReverseRowsStep::ReverseRowsStep(const DataStream & input_stream_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) +{ +} + +void ReverseRowsStep::transformPipeline(QueryPipeline & pipeline) +{ + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header); + }); +} + +} diff --git a/src/Processors/QueryPlan/ReverseRowsStep.h b/src/Processors/QueryPlan/ReverseRowsStep.h new file mode 100644 index 00000000000..955d022cde0 --- /dev/null +++ b/src/Processors/QueryPlan/ReverseRowsStep.h @@ -0,0 +1,18 @@ +#pragma once +#include + +namespace DB +{ + +/// Reverse rows in chunk. +class ReverseRowsStep : public ITransformingStep +{ +public: + ReverseRowsStep(const DataStream & input_stream_); + + String getName() const override { return "ReverseRows"; } + + void transformPipeline(QueryPipeline & pipeline) override; +}; + +} diff --git a/src/Processors/Transforms/AddingConstColumnTransform.h b/src/Processors/Transforms/AddingConstColumnTransform.h index 26d70d27ca7..15e9addd924 100644 --- a/src/Processors/Transforms/AddingConstColumnTransform.h +++ b/src/Processors/Transforms/AddingConstColumnTransform.h @@ -4,33 +4,40 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /// Adds a materialized const column to the chunk with a specified value. -template class AddingConstColumnTransform : public ISimpleTransform { public: - AddingConstColumnTransform(const Block & header, DataTypePtr data_type_, T value_, const String & column_name_) - : ISimpleTransform(header, addColumn(header, data_type_, column_name_), false) - , data_type(std::move(data_type_)), value(value_) {} + AddingConstColumnTransform(const Block & header, ColumnWithTypeAndName column_) + : ISimpleTransform(header, transformHeader(header, column_), false) + , column(std::move(column_)) + { + if (!column.column || !isColumnConst(*column.column) || !column.column->empty()) + throw Exception("AddingConstColumnTransform expected empty const column", ErrorCodes::LOGICAL_ERROR); + } String getName() const override { return "AddingConstColumnTransform"; } + static Block transformHeader(Block header, ColumnWithTypeAndName & column_) + { + header.insert(column_); + return header; + } + protected: void transform(Chunk & chunk) override { auto num_rows = chunk.getNumRows(); - chunk.addColumn(data_type->createColumnConst(num_rows, value)->convertToFullColumnIfConst()); + chunk.addColumn(column.column->cloneResized(num_rows)->convertToFullColumnIfConst()); } private: - static Block addColumn(Block header, const DataTypePtr & data_type, const String & column_name) - { - header.insert({data_type->createColumn(), data_type, column_name}); - return header; - } - - DataTypePtr data_type; - T value; + ColumnWithTypeAndName column; }; } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 8c3709f2d34..972e4995747 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -90,6 +90,7 @@ SRCS( Port.cpp printPipeline.cpp QueryPipeline.cpp + QueryPlan/AddingConstColumnStep.cpp QueryPlan/AddingDelayedSourceStep.cpp QueryPlan/AddingMissedStep.cpp QueryPlan/AggregatingStep.cpp @@ -111,12 +112,14 @@ SRCS( QueryPlan/MaterializingStep.cpp QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp + QueryPlan/MergingFinal.cpp QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp QueryPlan/PartialSortingStep.cpp QueryPlan/QueryPlan.cpp QueryPlan/ReadFromPreparedSource.cpp QueryPlan/ReadNothingStep.cpp + QueryPlan/ReverseRowsStep.cpp QueryPlan/RollupStep.cpp QueryPlan/SettingQuotaAndLimitsStep.cpp QueryPlan/TotalsHavingStep.cpp diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 8df832a9511..2260587b432 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -49,6 +49,7 @@ using Processors = std::vector; class Pipe; class QueryPlan; +using QueryPlanPtr = std::unique_ptr; class StoragePolicy; using StoragePolicyPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8b5b337bcec..72940e2754b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -20,6 +20,16 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include /// Allow to use __uint128_t as a template parameter for boost::rational. // https://stackoverflow.com/questions/41198673/uint128-t-not-working-with-clang-and-libstdc @@ -46,20 +56,6 @@ namespace std #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include namespace ProfileEvents @@ -151,7 +147,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz } -Pipe MergeTreeDataSelectExecutor::read( +QueryPlanPtr MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -166,7 +162,7 @@ Pipe MergeTreeDataSelectExecutor::read( max_block_numbers_to_read); } -Pipe MergeTreeDataSelectExecutor::readFromParts( +QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, @@ -688,7 +684,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); - Pipe res; + QueryPlanPtr plan; /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. @@ -709,7 +705,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( std::sort(column_names_to_read.begin(), column_names_to_read.end()); column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); - res = spreadMarkRangesAmongStreamsFinal( + plan = spreadMarkRangesAmongStreamsFinal( std::move(parts_with_ranges), num_streams, column_names_to_read, @@ -731,7 +727,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActions(false); - res = spreadMarkRangesAmongStreamsWithOrder( + plan = spreadMarkRangesAmongStreamsWithOrder( std::move(parts_with_ranges), num_streams, column_names_to_read, @@ -747,7 +743,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( } else { - res = spreadMarkRangesAmongStreams( + plan = spreadMarkRangesAmongStreams( std::move(parts_with_ranges), num_streams, column_names_to_read, @@ -762,40 +758,47 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - res.addSimpleTransform([&filter_expression, &filter_function](const Block & header) - { - return std::make_shared( - header, filter_expression, filter_function->getColumnName(), false); - }); + auto sampling_step = std::make_unique( + plan->getCurrentDataStream(), + filter_expression, + filter_function->getColumnName(), + false); + + sampling_step->setStepDescription("Sampling"); + plan->addStep(std::move(sampling_step)); } if (result_projection) { - res.addSimpleTransform([&result_projection](const Block & header) - { - return std::make_shared(header, result_projection); - }); + auto projection_step = std::make_unique(plan->getCurrentDataStream(), result_projection); + projection_step->setStepDescription("Remove unused columns after reading from storage"); + plan->addStep(std::move(projection_step)); } /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. if (sample_factor_column_queried) { - res.addSimpleTransform([used_sample_factor](const Block & header) - { - return std::make_shared>( - header, std::make_shared(), used_sample_factor, "_sample_factor"); - }); + ColumnWithTypeAndName column; + column.name = "_sample_factor"; + column.type = std::make_shared(); + column.column = column.type->createColumnConst(0, Field(used_sample_factor)); + + auto adding_column = std::make_unique(plan->getCurrentDataStream(), std::move(column)); + adding_column->setStepDescription("Add _sample_factor column"); + plan->addStep(std::move(adding_column)); } if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - res.addSimpleTransform([&query_info](const Block & header) - { - return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); - }); + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + query_info.prewhere_info->remove_columns_actions); + + expression_step->setStepDescription("Remove unused columns after PREWHERE"); + plan->addStep(std::move(expression_step)); } - return res; + return plan; } namespace @@ -820,8 +823,20 @@ size_t roundRowsOrBytesToMarks( } +static QueryPlanPtr createPlanFromPipe(Pipe pipe, const std::string & description = "") +{ + auto plan = std::make_unique(); -Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( + std::string storage_name = "MergeTree"; + if (!description.empty()) + storage_name += ' ' + description; + + auto step = std::make_unique(std::move(pipe), storage_name); + plan->addStep(std::move(step)); + return plan; +} + +QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -915,7 +930,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( res.emplace_back(std::move(source)); } - return Pipe::unitePipes(std::move(res)); + return createPlanFromPipe(Pipe::unitePipes(std::move(res))); } else { @@ -939,19 +954,18 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( if (pipe.numOutputPorts() > 1) pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts())); - return pipe; + return createPlanFromPipe(std::move(pipe)); } } -static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data) +static ExpressionActionsPtr createProjection(const Block & header, const MergeTreeData & data) { - const auto & header = pipe.getHeader(); auto projection = std::make_shared(header.getNamesAndTypesList(), data.global_context); projection->add(ExpressionAction::project(header.getNames())); return projection; } -Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( +QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -1053,6 +1067,8 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); + std::vector plans; + for (size_t i = 0; i < num_streams && !parts.empty(); ++i) { size_t need_marks = min_marks_per_stream; @@ -1151,17 +1167,15 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( } } - auto pipe = Pipe::unitePipes(std::move(pipes)); + auto plan = createPlanFromPipe(Pipe::unitePipes(std::move(pipes)), " with order"); if (input_order_info->direction != 1) { - pipe.addSimpleTransform([](const Block & header) - { - return std::make_shared(header); - }); + auto reverse_step = std::make_unique(plan->getCurrentDataStream()); + plan->addStep(std::move(reverse_step)); } - if (pipe.numOutputPorts() > 1 && need_preliminary_merge) + if (need_preliminary_merge) { SortDescription sort_description; for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) @@ -1169,24 +1183,45 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( input_order_info->direction, 1); /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) - { - return std::make_shared(header, sorting_key_prefix_expr); - }); + out_projection = createProjection(plan->getCurrentDataStream().header, data); - pipe.addTransform(std::make_shared( - pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + sorting_key_prefix_expr); + + expression_step->setStepDescription("Calculate sorting key prefix"); + plan->addStep(std::move(expression_step)); + + auto merging_sorted = std::make_unique( + plan->getCurrentDataStream(), + sort_description, + max_block_size); + + merging_sorted->setStepDescription("Merge sorting mark ranges"); + plan->addStep(std::move(merging_sorted)); } - res.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan)); } - return Pipe::unitePipes(std::move(res)); + if (plans.size() == 1) + return std::move(plans.front()); + + DataStreams input_streams; + for (const auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + const auto & common_header = plans.front()->getCurrentDataStream().header; + auto union_step = std::make_unique(std::move(input_streams), common_header); + + auto plan = std::make_unique(); + plan->unitePlans(std::move(union_step), std::move(plans)); + + return plan; } -Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( +QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -1224,7 +1259,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - Pipe pipe; + QueryPlanPtr plan; { Pipes pipes; @@ -1241,17 +1276,21 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( pipes.emplace_back(std::move(source_processor)); } - pipe = Pipe::unitePipes(std::move(pipes)); + auto pipe = Pipe::unitePipes(std::move(pipes)); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader(), data); + + plan = createPlanFromPipe(std::move(pipe), "with final"); } - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(pipe, data); + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + metadata_snapshot->getSortingKey().expression); - pipe.addSimpleTransform([&metadata_snapshot](const Block & header) - { - return std::make_shared(header, metadata_snapshot->getSortingKey().expression); - }); + expression_step->setStepDescription("Calculate sorting key expression"); + plan->addStep(std::move(expression_step)); Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; @@ -1260,108 +1299,25 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - Block header = pipe.getHeader(); + const auto & header = plan->getCurrentDataStream().header; for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - auto get_merging_processor = [&]() -> MergingTransformPtr - { - switch (data.merging_params.mode) - { - case MergeTreeData::MergingParams::Ordinary: - { - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, max_block_size); - } - - case MergeTreeData::MergingParams::Collapsing: - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, data.merging_params.sign_column, true, max_block_size); - - case MergeTreeData::MergingParams::Summing: - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, data.merging_params.columns_to_sum, partition_key_columns, max_block_size); - - case MergeTreeData::MergingParams::Aggregating: - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, max_block_size); - - case MergeTreeData::MergingParams::Replacing: - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, data.merging_params.version_column, max_block_size); - - case MergeTreeData::MergingParams::VersionedCollapsing: - return std::make_shared(header, pipe.numOutputPorts(), - sort_description, data.merging_params.sign_column, max_block_size); - - case MergeTreeData::MergingParams::Graphite: - throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); - } - - __builtin_unreachable(); - }; - if (num_streams > settings.max_final_threads) num_streams = settings.max_final_threads; - if (num_streams <= 1 || sort_description.empty()) - { - pipe.addTransform(get_merging_processor()); - return pipe; - } + auto final_step = std::make_unique( + plan->getCurrentDataStream(), + num_streams, + sort_description, + data.merging_params, + partition_key_columns, + max_block_size); - ColumnNumbers key_columns; - key_columns.reserve(sort_description.size()); + final_step->setStepDescription("Merge rows for FINAL"); + plan->addStep(std::move(final_step)); - for (auto & desc : sort_description) - { - if (!desc.column_name.empty()) - key_columns.push_back(header.getPositionByName(desc.column_name)); - else - key_columns.emplace_back(desc.column_number); - } - - pipe.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, num_streams, key_columns); - }); - - pipe.transform([&](OutputPortRawPtrs ports) - { - Processors processors; - std::vector output_ports; - processors.reserve(ports.size() + num_streams); - output_ports.reserve(ports.size()); - - for (auto & port : ports) - { - auto copier = std::make_shared(header, num_streams); - connect(*port, copier->getInputPort()); - output_ports.emplace_back(copier->getOutputs().begin()); - processors.emplace_back(std::move(copier)); - } - - for (size_t i = 0; i < num_streams; ++i) - { - auto merge = get_merging_processor(); - merge->setSelectorPosition(i); - auto input = merge->getInputs().begin(); - - /// Connect i-th merge with i-th input port of every copier. - for (size_t j = 0; j < ports.size(); ++j) - { - connect(*output_ports[j], *input); - ++output_ports[j]; - ++input; - } - - processors.emplace_back(std::move(merge)); - } - - return processors; - }); - - return pipe; + return plan; } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 5894d6e044b..7fd0e580b79 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -24,7 +24,7 @@ public: */ using PartitionIdToMaxBlock = std::unordered_map; - Pipe read( + QueryPlanPtr read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -33,7 +33,7 @@ public: unsigned num_streams, const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; - Pipe readFromParts( + QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -48,7 +48,7 @@ private: Poco::Logger * log; - Pipe spreadMarkRangesAmongStreams( + QueryPlanPtr spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -61,7 +61,7 @@ private: const MergeTreeReaderSettings & reader_settings) const; /// out_projection - save projection only with columns, requested to read - Pipe spreadMarkRangesAmongStreamsWithOrder( + QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -75,7 +75,7 @@ private: const MergeTreeReaderSettings & reader_settings, ExpressionActionsPtr & out_projection) const; - Pipe spreadMarkRangesAmongStreamsFinal( + QueryPlanPtr spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & column_names, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index c13f540ad34..049b4a99084 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -27,8 +29,11 @@ public: size_t max_block_size, unsigned num_streams) override { - return MergeTreeDataSelectExecutor(part->storage) - .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams); + QueryPlan query_plan = + std::move(*MergeTreeDataSelectExecutor(part->storage) + .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); + + return QueryPipeline::getPipe(std::move(*query_plan.buildQueryPipeline())); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 42ac9cb6a5b..e660c27dd6b 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -319,10 +319,13 @@ Pipe StorageMerge::createSources( if (has_table_virtual_column) { - pipe.addSimpleTransform([name = table_name](const Block & stream_header) + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(); + column.column = column.type->createColumnConst(0, Field(table_name)); + pipe.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared>( - stream_header, std::make_shared(), name, "_table"); + return std::make_shared(stream_header, column); }); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 55fb42b550e..35b147f22f7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -171,17 +171,31 @@ StorageMergeTree::~StorageMergeTree() shutdown(); } -Pipe StorageMergeTree::read( +void StorageMergeTree::read( + QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned num_streams) +{ + query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); +} + +Pipe StorageMergeTree::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, const size_t max_block_size, const unsigned num_streams) { - return reader.read(column_names, metadata_snapshot, query_info, - context, max_block_size, num_streams); + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); } std::optional StorageMergeTree::totalRows() const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..c890d5f3fc2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -46,6 +46,16 @@ public: size_t max_block_size, unsigned num_streams) override; + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + std::optional totalRows() const override; std::optional totalBytes() const override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9613bd5111d..8e2ecf1f189 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3514,7 +3514,9 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg return max_added_blocks; } -Pipe StorageReplicatedMergeTree::read( + +void StorageReplicatedMergeTree::read( + QueryPlan & query_plan, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -3531,10 +3533,25 @@ Pipe StorageReplicatedMergeTree::read( if (context.getSettingsRef().select_sequential_consistency) { auto max_added_blocks = getMaxAddedBlocks(); - return reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams, &max_added_blocks); + query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams, &max_added_blocks)); + return; } - return reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams); + query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); +} + +Pipe StorageReplicatedMergeTree::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const unsigned num_streams) +{ + QueryPlan plan; + read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d851082d5c2..2cea4817627 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -96,6 +96,16 @@ public: size_t max_block_size, unsigned num_streams) override; + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + std::optional totalRows() const override; std::optional totalBytes() const override; From f02ad3dc8ecf6f3228faa667364823ef993ca356 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Oct 2020 21:02:22 +0300 Subject: [PATCH 14/54] Use QueryPlan while reading from MergeTree. --- src/Processors/QueryPlan/MergingFinal.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp index cd9d6abb4c3..cde4d99ea47 100644 --- a/src/Processors/QueryPlan/MergingFinal.cpp +++ b/src/Processors/QueryPlan/MergingFinal.cpp @@ -90,7 +90,7 @@ void MergingFinal::transformPipeline(QueryPipeline & pipeline) __builtin_unreachable(); }; - if (num_outputs <= 1 || sort_description.empty()) + if (num_output_streams <= 1 || sort_description.empty()) { pipeline.addTransform(get_merging_processor()); return; @@ -109,25 +109,25 @@ void MergingFinal::transformPipeline(QueryPipeline & pipeline) pipeline.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(stream_header, num_outputs, key_columns); + return std::make_shared(stream_header, num_output_streams, key_columns); }); pipeline.transform([&](OutputPortRawPtrs ports) { Processors processors; std::vector output_ports; - processors.reserve(ports.size() + num_outputs); + processors.reserve(ports.size() + num_output_streams); output_ports.reserve(ports.size()); for (auto & port : ports) { - auto copier = std::make_shared(header, num_outputs); + auto copier = std::make_shared(header, num_output_streams); connect(*port, copier->getInputPort()); output_ports.emplace_back(copier->getOutputs().begin()); processors.emplace_back(std::move(copier)); } - for (size_t i = 0; i < num_outputs; ++i) + for (size_t i = 0; i < num_output_streams; ++i) { auto merge = get_merging_processor(); merge->setSelectorPosition(i); From 513402814069d1c2a300359825bba0eb7ba52850 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Oct 2020 21:19:38 +0300 Subject: [PATCH 15/54] Fix style. --- src/Processors/QueryPlan/MergingFinal.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp index cde4d99ea47..9a33ce7709e 100644 --- a/src/Processors/QueryPlan/MergingFinal.cpp +++ b/src/Processors/QueryPlan/MergingFinal.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static ITransformingStep::Traits getTraits() { return ITransformingStep::Traits From ea131989befee7e638a294e98551f7d60bc33d3a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Oct 2020 21:47:20 +0300 Subject: [PATCH 16/54] Try fix test. --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 72940e2754b..3129fa81be9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -540,7 +540,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (no_data) { LOG_DEBUG(log, "Sampling yields no data."); - return {}; + return std::make_unique(); } LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); @@ -678,7 +678,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( LOG_DEBUG(log, "Selected {} parts by partition key, {} parts by primary key, {} marks by primary key, {} marks to read from {} ranges", parts.size(), parts_with_ranges.size(), sum_marks_pk.load(std::memory_order_relaxed), sum_marks, sum_ranges); if (parts_with_ranges.empty()) - return {}; + return std::make_unique(); ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); @@ -756,6 +756,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( reader_settings); } + if (!plan) + return std::make_unique(); + if (use_sampling) { auto sampling_step = std::make_unique( From 5ea4dc0850189a76e531883bdeabb9c22a32175f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Oct 2020 14:25:16 +0300 Subject: [PATCH 17/54] Try fix tests. --- src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 049b4a99084..769e82155b9 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -33,6 +33,9 @@ public: std::move(*MergeTreeDataSelectExecutor(part->storage) .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); + if (!query_plan.isInitialized()) + return {}; + return QueryPipeline::getPipe(std::move(*query_plan.buildQueryPipeline())); } From bef96faa1a062af6ae6c9968bc859850f54dd2db Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Oct 2020 15:38:33 +0300 Subject: [PATCH 18/54] Tru fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5a6bb5770cf..f8e896b3448 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1382,7 +1382,7 @@ void InterpreterSelectQuery::executeFetchColumns( ErrorCodes::TOO_MANY_COLUMNS); /// General limit for the number of threads. - query_plan.setMaxThreads(settings.max_threads); + size_t max_threads_execute_query = settings.max_threads; /** With distributed query processing, almost no computations are done in the threads, * but wait and receive data from remote servers. @@ -1395,8 +1395,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (storage && storage->isRemote()) { is_remote = true; - max_streams = settings.max_distributed_connections; - query_plan.setMaxThreads(max_streams); + max_threads_execute_query = max_streams = settings.max_distributed_connections; } UInt64 max_block_size = settings.max_block_size; @@ -1421,8 +1420,7 @@ void InterpreterSelectQuery::executeFetchColumns( && limit_length + limit_offset < max_block_size) { max_block_size = std::max(UInt64(1), limit_length + limit_offset); - max_streams = 1; - query_plan.setMaxThreads(max_streams); + max_threads_execute_query = max_streams = 1; } if (!max_block_size) @@ -1529,6 +1527,8 @@ void InterpreterSelectQuery::executeFetchColumns( else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); + query_plan.setMaxThreads(max_threads_execute_query); + /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { From 11c0c2864fb539d138b4c96ea85e63b58921a383 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Oct 2020 17:23:06 +0300 Subject: [PATCH 19/54] Fix build. --- src/Processors/QueryPlan/MergingFinal.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp index 9a33ce7709e..c1dec231f11 100644 --- a/src/Processors/QueryPlan/MergingFinal.cpp +++ b/src/Processors/QueryPlan/MergingFinal.cpp @@ -119,9 +119,9 @@ void MergingFinal::transformPipeline(QueryPipeline & pipeline) pipeline.transform([&](OutputPortRawPtrs ports) { - Processors processors; + Processors transforms; std::vector output_ports; - processors.reserve(ports.size() + num_output_streams); + transforms.reserve(ports.size() + num_output_streams); output_ports.reserve(ports.size()); for (auto & port : ports) @@ -129,7 +129,7 @@ void MergingFinal::transformPipeline(QueryPipeline & pipeline) auto copier = std::make_shared(header, num_output_streams); connect(*port, copier->getInputPort()); output_ports.emplace_back(copier->getOutputs().begin()); - processors.emplace_back(std::move(copier)); + transforms.emplace_back(std::move(copier)); } for (size_t i = 0; i < num_output_streams; ++i) @@ -146,10 +146,10 @@ void MergingFinal::transformPipeline(QueryPipeline & pipeline) ++input; } - processors.emplace_back(std::move(merge)); + transforms.emplace_back(std::move(merge)); } - return processors; + return transforms; }); } From 5bbae0953a748402aceab3e09566b3a7defaf8e1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Oct 2020 09:53:51 +0300 Subject: [PATCH 20/54] Fix order of resource destruction in SettingQuotaAndLimitsStep. --- src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp | 8 +++++--- src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h | 2 +- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 5b05ad77d6c..588dd7599a1 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -30,20 +30,18 @@ SettingQuotaAndLimitsStep::SettingQuotaAndLimitsStep( std::shared_ptr quota_, std::shared_ptr context_) : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + , context(std::move(context_)) , storage(std::move(storage_)) , table_lock(std::move(table_lock_)) , limits(limits_) , leaf_limits(leaf_limits_) , quota(std::move(quota_)) - , context(std::move(context_)) { } void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline) { /// Table lock is stored inside pipeline here. - pipeline.addTableLock(table_lock); - pipeline.setLimits(limits); /** @@ -59,11 +57,15 @@ void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline) if (quota) pipeline.setQuota(quota); + /// Order of resources below is important. if (context) pipeline.addInterpreterContext(std::move(context)); if (storage) pipeline.addStorageHolder(std::move(storage)); + + if (table_lock) + pipeline.addTableLock(std::move(table_lock)); } } diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h index 7ec4cfa91c6..66e44e18cd4 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -33,12 +33,12 @@ public: void transformPipeline(QueryPipeline & pipeline) override; private: + std::shared_ptr context; StoragePtr storage; TableLockHolder table_lock; StreamLocalLimits limits; SizeLimits leaf_limits; std::shared_ptr quota; - std::shared_ptr context; }; } From d968f276b5f6f0e6d5b85e968b5ead503649ba38 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Oct 2020 11:21:05 +0300 Subject: [PATCH 21/54] Fix buffer table. --- src/Storages/StorageBuffer.cpp | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 162463a4e33..f0fac1e4949 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -237,20 +237,24 @@ void StorageBuffer::read( query_plan, columns_intersection, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - auto adding_missed = std::make_unique( - query_plan.getCurrentDataStream(), - header_after_adding_defaults, - metadata_snapshot->getColumns().getDefaults(), context); + if (query_plan.isInitialized()) + { - adding_missed->setStepDescription("Add columns missing in destination table"); - query_plan.addStep(std::move(adding_missed)); + auto adding_missed = std::make_unique( + query_plan.getCurrentDataStream(), + header_after_adding_defaults, + metadata_snapshot->getColumns().getDefaults(), context); - auto converting = std::make_unique( - query_plan.getCurrentDataStream(), - header); + adding_missed->setStepDescription("Add columns missing in destination table"); + query_plan.addStep(std::move(adding_missed)); - converting->setStepDescription("Convert destination table columns to Buffer table structure"); - query_plan.addStep(std::move(converting)); + auto converting = std::make_unique( + query_plan.getCurrentDataStream(), + header); + + converting->setStepDescription("Convert destination table columns to Buffer table structure"); + query_plan.addStep(std::move(converting)); + } } } From efd1f10a6d8bff23132c1b20799ca62810c21b82 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Oct 2020 11:55:03 +0300 Subject: [PATCH 22/54] Fix build. --- src/Processors/QueryPlan/AddingMissedStep.cpp | 6 +++--- src/Processors/QueryPlan/AddingMissedStep.h | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/AddingMissedStep.cpp b/src/Processors/QueryPlan/AddingMissedStep.cpp index f2d06c033b1..d354405b5ac 100644 --- a/src/Processors/QueryPlan/AddingMissedStep.cpp +++ b/src/Processors/QueryPlan/AddingMissedStep.cpp @@ -25,10 +25,10 @@ static ITransformingStep::Traits getTraits() AddingMissedStep::AddingMissedStep( const DataStream & input_stream_, Block result_header_, - const ColumnDefaults & column_defaults_, + ColumnsDescription columns_, const Context & context_) : ITransformingStep(input_stream_, result_header_, getTraits()) - , column_defaults(column_defaults_) + , columns(std::move(columns_)) , context(context_) { updateDistinctColumns(output_stream->header, output_stream->distinct_columns); @@ -38,7 +38,7 @@ void AddingMissedStep::transformPipeline(QueryPipeline & pipeline) { pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, output_stream->header, column_defaults, context); + return std::make_shared(header, output_stream->header, columns, context); }); } diff --git a/src/Processors/QueryPlan/AddingMissedStep.h b/src/Processors/QueryPlan/AddingMissedStep.h index 8e3b8148b6a..ce755b79fdf 100644 --- a/src/Processors/QueryPlan/AddingMissedStep.h +++ b/src/Processors/QueryPlan/AddingMissedStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { @@ -11,7 +11,7 @@ class AddingMissedStep : public ITransformingStep public: AddingMissedStep(const DataStream & input_stream_, Block result_header_, - const ColumnDefaults & column_defaults_, + ColumnsDescription columns_, const Context & context_); String getName() const override { return "AddingMissed"; } @@ -19,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; private: - ColumnDefaults column_defaults; + ColumnsDescription columns; const Context & context; }; From 7caf6da363525a67c1eecfe421a1a20ec9e27229 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Oct 2020 11:56:21 +0300 Subject: [PATCH 23/54] Fix build. --- src/Storages/StorageBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f0fac1e4949..182087c24ad 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -243,7 +243,7 @@ void StorageBuffer::read( auto adding_missed = std::make_unique( query_plan.getCurrentDataStream(), header_after_adding_defaults, - metadata_snapshot->getColumns().getDefaults(), context); + metadata_snapshot->getColumns(), context); adding_missed->setStepDescription("Add columns missing in destination table"); query_plan.addStep(std::move(adding_missed)); From c5cb05f5f33238d7561e953e9bca1dc494e5e1d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Oct 2020 14:26:29 +0300 Subject: [PATCH 24/54] Try fix tests. --- src/Processors/QueryPlan/QueryPlan.cpp | 11 +++++++++++ src/Processors/QueryPlan/QueryPlan.h | 5 +++++ src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 5 +---- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageView.cpp | 2 +- 9 files changed, 23 insertions(+), 10 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 1ff844480a9..e681d4989d3 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -185,6 +185,17 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline() return last_pipeline; } +Pipe QueryPlan::convertToPipe() +{ + if (!isInitialized()) + return {}; + + if (isCompleted()) + throw Exception("Cannot convert completed QueryPlan to Pipe", ErrorCodes::LOGICAL_ERROR); + + return QueryPipeline::getPipe(std::move(*buildQueryPipeline())); +} + void QueryPlan::addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 4f558e04c55..8aa0e868dc6 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -20,6 +20,8 @@ class WriteBuffer; class QueryPlan; using QueryPlanPtr = std::unique_ptr; +class Pipe; + /// A tree of query steps. /// The goal of QueryPlan is to build QueryPipeline. /// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimisations. @@ -42,6 +44,9 @@ public: QueryPipelinePtr buildQueryPipeline(); + /// If initialized, build pipeline and convert to pipe. Otherwise, return empty pipe. + Pipe convertToPipe(); + struct ExplainPlanOptions { /// Add output header to step. diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 769e82155b9..7c30c7302b3 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -33,10 +33,7 @@ public: std::move(*MergeTreeDataSelectExecutor(part->storage) .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); - if (!query_plan.isInitialized()) - return {}; - - return QueryPipeline::getPipe(std::move(*query_plan.buildQueryPipeline())); + return query_plan.convertToPipe(); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 182087c24ad..f6e728ebbed 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -160,7 +160,7 @@ Pipe StorageBuffer::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } void StorageBuffer::read( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 6e7d7ea41bd..0b3070c88f6 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -505,7 +505,7 @@ Pipe StorageDistributed::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } void StorageDistributed::read( diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 69669c0b680..9ea4a5d91d8 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -117,7 +117,7 @@ Pipe StorageMaterializedView::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } void StorageMaterializedView::read( diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 35b147f22f7..6c0b082c6a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -195,7 +195,7 @@ Pipe StorageMergeTree::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } std::optional StorageMergeTree::totalRows() const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8e2ecf1f189..ef06d27101b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3551,7 +3551,7 @@ Pipe StorageReplicatedMergeTree::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index e71228f2a23..949e7930ccf 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -61,7 +61,7 @@ Pipe StorageView::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline())); + return plan.convertToPipe(); } void StorageView::read( From d94d88a6c02fab3384f63d0c796504c55555db5a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Oct 2020 15:04:15 +0300 Subject: [PATCH 25/54] Fix style. --- src/Processors/QueryPlan/ReadFromStorageStep.cpp | 0 src/Processors/QueryPlan/ReadFromStorageStep.h | 0 2 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 src/Processors/QueryPlan/ReadFromStorageStep.cpp delete mode 100644 src/Processors/QueryPlan/ReadFromStorageStep.h diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h deleted file mode 100644 index e69de29bb2d..00000000000 From 337098367bb80e38fc48f4cc55746c023f2502aa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Oct 2020 11:36:15 +0300 Subject: [PATCH 26/54] Try fix perf --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fc496ce9dc3..9ba0b4668fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1424,6 +1424,8 @@ void InterpreterSelectQuery::executeFetchColumns( if (!max_block_size) throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND); + query_plan.setMaxThreads(max_threads_execute_query); + /// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input? if (query_plan.isInitialized()) { @@ -1525,8 +1527,6 @@ void InterpreterSelectQuery::executeFetchColumns( else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); - query_plan.setMaxThreads(max_threads_execute_query); - /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { From c91b45363055f3ed7f690c7c383aa01dc204d9cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Oct 2020 14:08:12 +0300 Subject: [PATCH 27/54] Try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++-- src/Processors/QueryPlan/QueryPlan.h | 1 + 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2a4005e2acc..465dfb0c52c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1439,8 +1439,6 @@ void InterpreterSelectQuery::executeFetchColumns( if (!max_block_size) throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND); - query_plan.setMaxThreads(max_threads_execute_query); - /// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input? if (query_plan.isInitialized()) { @@ -1542,6 +1540,10 @@ void InterpreterSelectQuery::executeFetchColumns( else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); + /// Specify the number of threads only if it wasn't specified in storage. + if (!query_plan.getMaxThreads()) + query_plan.setMaxThreads(max_threads_execute_query); + /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 8aa0e868dc6..d53193a7aa5 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -69,6 +69,7 @@ public: /// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines. /// TODO: make it in a better way. void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } + size_t getMaxThreads() const { return max_threads; } void addInterpreterContext(std::shared_ptr context); From a42fd18390bd0011516302f391721aec5d019947 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Oct 2020 15:01:22 +0300 Subject: [PATCH 28/54] Try fix tests. --- src/Storages/StorageMaterializedView.cpp | 29 +++++++++++++----------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index c07f9476a15..b587d4bc94c 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -140,21 +140,24 @@ void StorageMaterializedView::read( storage->read(query_plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - StreamLocalLimits limits; - SizeLimits leaf_limits; + if (query_plan.isInitialized()) + { + StreamLocalLimits limits; + SizeLimits leaf_limits; - /// Add table lock for destination table. - auto adding_limits_and_quota = std::make_unique( - query_plan.getCurrentDataStream(), - storage, - std::move(lock), - limits, - leaf_limits, - nullptr, - nullptr); + /// Add table lock for destination table. + auto adding_limits_and_quota = std::make_unique( + query_plan.getCurrentDataStream(), + storage, + std::move(lock), + limits, + leaf_limits, + nullptr, + nullptr); - adding_limits_and_quota->setStepDescription("Lock destination table for Buffer"); - query_plan.addStep(std::move(adding_limits_and_quota)); + adding_limits_and_quota->setStepDescription("Lock destination table for Buffer"); + query_plan.addStep(std::move(adding_limits_and_quota)); + } } BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) From c73c801825532e026ffa1908c800aca645e4de95 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Mon, 9 Nov 2020 17:49:48 +0300 Subject: [PATCH 29/54] Set sane default value for metric_log's collect inverval Otherwise server will crash on start if config value is missing. No test since we can't test configs. --- src/Interpreters/SystemLog.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 1e396a0fed1..01077602d90 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -25,6 +25,7 @@ namespace { constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; +constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config template @@ -125,7 +126,8 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi if (metric_log) { - size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds"); + size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds", + DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS); metric_log->startCollectMetric(collect_interval_milliseconds); } From dce9f26b024f845ec7ea02a1ce955921135be322 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 18:12:40 +0300 Subject: [PATCH 30/54] Empty commit. --- src/Storages/IStorage.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 3b04c6cd632..ddd5b6727dc 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -80,6 +80,7 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons return result; } + Pipe IStorage::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, From 3c6794bf37bc126751f67aeca11c97df75c7b707 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 9 Nov 2020 20:22:05 +0300 Subject: [PATCH 31/54] Add Floyd-Rivest selection algorithm instead of std::partial_sort --- CMakeLists.txt | 1 + cmake/find/miniselect.cmake | 2 + contrib/miniselect/.clang-format | 1 + contrib/miniselect/.gitignore | 100 ++ contrib/miniselect/.travis.yml | 140 +++ contrib/miniselect/AUTHORS | 2 + contrib/miniselect/CMakeLists.txt | 52 + contrib/miniselect/CONTRIBUTORS | 1 + contrib/miniselect/LICENSE_1_0.txt | 23 + contrib/miniselect/README.md | 272 +++++ contrib/miniselect/benches/bench_common.h | 170 ++++ .../miniselect/benches/benchmark_select.cpp | 46 + contrib/miniselect/benches/benchmark_sort.cpp | 46 + contrib/miniselect/examples/example.cpp | 18 + contrib/miniselect/fuzz/CMakeLists.txt | 38 + .../miniselect/fuzz/build_like_oss_fuzz.sh | 22 + contrib/miniselect/fuzz/fuzz_select.cpp | 66 ++ contrib/miniselect/fuzz/fuzz_sort.cpp | 69 ++ .../miniselect/fuzz/fuzz_string_select.cpp | 70 ++ contrib/miniselect/fuzz/fuzz_string_sort.cpp | 73 ++ contrib/miniselect/fuzz/main.cpp | 22 + contrib/miniselect/fuzz/ossfuzz.sh | 23 + .../include/miniselect/floyd_rivest_select.h | 120 +++ .../include/miniselect/median_of_3_random.h | 69 ++ .../include/miniselect/median_of_medians.h | 71 ++ .../include/miniselect/median_of_ninthers.h | 190 ++++ .../miniselect/include/miniselect/pdqselect.h | 935 ++++++++++++++++++ .../miniselect/private/median_common.h | 437 ++++++++ contrib/miniselect/testing/test_common.h | 180 ++++ contrib/miniselect/testing/test_select.cpp | 231 +++++ contrib/miniselect/testing/test_sort.cpp | 161 +++ src/CMakeLists.txt | 1 + src/Columns/ColumnArray.cpp | 5 +- src/Columns/ColumnDecimal.cpp | 9 +- src/Columns/ColumnDecimal.h | 5 +- src/Columns/ColumnFixedString.cpp | 9 +- src/Columns/ColumnLowCardinality.cpp | 3 +- src/Columns/ColumnString.cpp | 5 +- src/Columns/ColumnTuple.cpp | 4 +- src/Columns/ColumnVector.cpp | 10 +- 40 files changed, 3680 insertions(+), 22 deletions(-) create mode 100644 cmake/find/miniselect.cmake create mode 100644 contrib/miniselect/.clang-format create mode 100644 contrib/miniselect/.gitignore create mode 100644 contrib/miniselect/.travis.yml create mode 100644 contrib/miniselect/AUTHORS create mode 100644 contrib/miniselect/CMakeLists.txt create mode 100644 contrib/miniselect/CONTRIBUTORS create mode 100644 contrib/miniselect/LICENSE_1_0.txt create mode 100644 contrib/miniselect/README.md create mode 100644 contrib/miniselect/benches/bench_common.h create mode 100644 contrib/miniselect/benches/benchmark_select.cpp create mode 100644 contrib/miniselect/benches/benchmark_sort.cpp create mode 100644 contrib/miniselect/examples/example.cpp create mode 100644 contrib/miniselect/fuzz/CMakeLists.txt create mode 100755 contrib/miniselect/fuzz/build_like_oss_fuzz.sh create mode 100644 contrib/miniselect/fuzz/fuzz_select.cpp create mode 100644 contrib/miniselect/fuzz/fuzz_sort.cpp create mode 100644 contrib/miniselect/fuzz/fuzz_string_select.cpp create mode 100644 contrib/miniselect/fuzz/fuzz_string_sort.cpp create mode 100644 contrib/miniselect/fuzz/main.cpp create mode 100755 contrib/miniselect/fuzz/ossfuzz.sh create mode 100644 contrib/miniselect/include/miniselect/floyd_rivest_select.h create mode 100644 contrib/miniselect/include/miniselect/median_of_3_random.h create mode 100644 contrib/miniselect/include/miniselect/median_of_medians.h create mode 100644 contrib/miniselect/include/miniselect/median_of_ninthers.h create mode 100644 contrib/miniselect/include/miniselect/pdqselect.h create mode 100644 contrib/miniselect/include/miniselect/private/median_common.h create mode 100644 contrib/miniselect/testing/test_common.h create mode 100644 contrib/miniselect/testing/test_select.cpp create mode 100644 contrib/miniselect/testing/test_sort.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index 783a9f80b66..182d9989dc2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -445,6 +445,7 @@ include (cmake/find/brotli.cmake) include (cmake/find/protobuf.cmake) include (cmake/find/grpc.cmake) include (cmake/find/pdqsort.cmake) +include (cmake/find/miniselect.cmake) include (cmake/find/hdfs3.cmake) # uses protobuf include (cmake/find/poco.cmake) include (cmake/find/curl.cmake) diff --git a/cmake/find/miniselect.cmake b/cmake/find/miniselect.cmake new file mode 100644 index 00000000000..0a50c9bf4a8 --- /dev/null +++ b/cmake/find/miniselect.cmake @@ -0,0 +1,2 @@ +set(MINISELECT_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/miniselect/include) +message(STATUS "Using miniselect: ${MINISELECT_INCLUDE_DIR}") diff --git a/contrib/miniselect/.clang-format b/contrib/miniselect/.clang-format new file mode 100644 index 00000000000..f6cb8ad931f --- /dev/null +++ b/contrib/miniselect/.clang-format @@ -0,0 +1 @@ +BasedOnStyle: Google diff --git a/contrib/miniselect/.gitignore b/contrib/miniselect/.gitignore new file mode 100644 index 00000000000..f80f36759c8 --- /dev/null +++ b/contrib/miniselect/.gitignore @@ -0,0 +1,100 @@ +# eclipse project files +.cproject +.project +.settings + +# emacs temp files +*~ + +# vim temp files +.*.swp + +# XCode +^build/ +*.pbxuser +!default.pbxuser +*.mode1v3 +!default.mode1v3 +*.mode2v3 +!default.mode2v3 +*.perspectivev3 +!default.perspectivev3 +xcuserdata +*.xccheckout +*.moved-aside +DerivedData +*.hmap +*.ipa +*.xcuserstate +*.DS_Store + +# IDE specific folder for JetBrains IDEs +.idea/ +cmake-build-debug/ +cmake-build-release/ + +# Visual Studio Code artifacts +.vscode/* +.history/ + +# Visual Studio artifacts +/VS/ + +# C/C++ build outputs +.build/ +bins +gens +libs +objs + +# C++ ignore from https://github.com/github/gitignore/blob/master/C%2B%2B.gitignore + +# Prerequisites +*.d + +# Compiled Object files +*.slo +*.lo +*.o +*.obj + +# Precompiled Headers +*.gch +*.pch + +# Compiled Dynamic libraries +*.so +*.dylib +*.dll + +# Fortran module files +*.mod +*.smod + +# Compiled Static libraries +*.lai +*.la +*.a +*.lib + +# Executables +*.exe +*.out +*.app + + +# CMake files that may be specific to our installation + +# Build outputs +/build*/ +/visual_studio/ +/benchmark/ + +# Fuzzer outputs generated by instructions in fuzz/Fuzzing.md +/corpus.zip +/ossfuzz-out/ +/out/ + +# Generated docs +/doc/api +*.orig diff --git a/contrib/miniselect/.travis.yml b/contrib/miniselect/.travis.yml new file mode 100644 index 00000000000..a5036caf365 --- /dev/null +++ b/contrib/miniselect/.travis.yml @@ -0,0 +1,140 @@ +language: cpp + +dist: bionic + +matrix: + include: + - os: linux + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - g++-8 + env: + - COMPILER="CC=gcc-8 && CXX=g++-8" + compiler: gcc-8 + + - os: linux + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - g++-9 + env: + - COMPILER="CC=gcc-9 && CXX=g++-9" + compiler: gcc-9 + + - os: linux + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - g++-10 + env: + - COMPILER="CC=gcc-10 && CXX=g++-10" + compiler: gcc-10 + + - os: linux + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - g++-10 + env: + - COMPILER="CC=gcc-10 && CXX=g++-10" + - SANITIZE="on" + compiler: gcc-10-sanitize + + - os: linux + addons: + apt: + sources: + - llvm-toolchain-bionic-6.0 + packages: + - clang-6.0 + env: + - COMPILER="CC=clang-6.0 && CXX=clang++-6.0" + compiler: clang-6 + + - os: linux + addons: + apt: + sources: + - llvm-toolchain-bionic-7 + packages: + - clang-7 + env: + - COMPILER="CC=clang-7 && CXX=clang++-7" + compiler: clang-7 + + - os: linux + addons: + apt: + sources: + - llvm-toolchain-bionic-8 + packages: + - clang-8 + env: + - COMPILER="CC=clang-8 && CXX=clang++-8" + compiler: clang-8 + + - os: linux + addons: + apt: + sources: + - llvm-toolchain-bionic-9 + packages: + - clang-9 + env: + - COMPILER="CC=clang-9 && CXX=clang++-9" + compiler: clang-9 + + - os: linux + addons: + apt: + packages: + - clang-10 + sources: + - ubuntu-toolchain-r-test + - sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main' + key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' + env: + - COMPILER="CC=clang-10 && CXX=clang++-10" + compiler: clang-10 + + - os: linux + addons: + apt: + packages: + - clang-10 + sources: + - ubuntu-toolchain-r-test + - sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main' + key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' + env: + - COMPILER="CC=clang-10 && CXX=clang++-10" + - SANITIZE="on" + compiler: clang-10-sanitize + +before_install: + - eval "${COMPILER}" + - git clone https://github.com/google/benchmark.git + - git clone https://github.com/google/googletest.git benchmark/googletest + +install: + - export CMAKE_FLAGS="-DMINISELECT_TESTING=on -DCMAKE_BUILD_TYPE=RelWithDebInfo"; + - if [[ "${SANITIZE}" == "on" ]]; then + export CMAKE_FLAGS="${CMAKE_FLAGS} -DMINISELECT_SANITIZE=on"; + fi + - export CTEST_FLAGS="-j4 --output-on-failure -E checkperf" + +script: + - mkdir build + - cd build + - cmake $CMAKE_FLAGS .. + - cmake --build . -- -j2 + - ctest $CTEST_FLAGS diff --git a/contrib/miniselect/AUTHORS b/contrib/miniselect/AUTHORS new file mode 100644 index 00000000000..896a8046a73 --- /dev/null +++ b/contrib/miniselect/AUTHORS @@ -0,0 +1,2 @@ +# List of authors for copyright purposes, in no particular order +Danila Kutenin diff --git a/contrib/miniselect/CMakeLists.txt b/contrib/miniselect/CMakeLists.txt new file mode 100644 index 00000000000..09e92031784 --- /dev/null +++ b/contrib/miniselect/CMakeLists.txt @@ -0,0 +1,52 @@ +cmake_minimum_required(VERSION 3.7) +project(miniselect) + +option(MINISELECT_TESTING "Building the tests." OFF) +option(MINISELECT_SANITIZE "Building the library with sanitizers." OFF) +option(MINISELECT_BUILD_LIBCXX "Building the library with libcxx." OFF) +option(MINISELECT_ENABLE_FUZZING "Building the library with fuzzing." OFF) + +include_directories(include) + +if (MINISELECT_TESTING) + enable_testing() + set(CMAKE_CXX_STANDARD 17) + if (NOT CMAKE_BUILD_TYPE) + message(STATUS "No build type selected, default to Release") + set(CMAKE_BUILD_TYPE "Release") + endif() + if (MINISELECT_SANITIZE) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsanitize=address -fno-omit-frame-pointer -fsanitize=undefined -fno-sanitize-recover=all") + endif() + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -g -Wall -Wextra -Wpedantic -Wno-gnu-zero-variadic-macro-arguments") + + if (MINISELECT_BUILD_LIBCXX AND "${CMAKE_CXX_COMPILER_ID}" STREQUAL "Clang") + message(STATUS "Using libcxx as a default standard C++ library") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + endif() + + add_subdirectory(benchmark) + include_directories(testing) + include_directories(benches) + + add_executable(benchmark_sort benches/benchmark_sort.cpp) + target_link_libraries(benchmark_sort benchmark::benchmark gtest) + add_executable(benchmark_select benches/benchmark_select.cpp) + target_link_libraries(benchmark_select benchmark::benchmark gtest) + + set(TEST_SOURCES testing/test_select.cpp) + add_executable(test_select ${TEST_SOURCES}) + target_link_libraries(test_select gtest gmock gtest_main) + add_test(NAME test_select COMMAND test_select) + + set(TEST_SOURCES testing/test_sort.cpp) + add_executable(test_sort ${TEST_SOURCES}) + target_link_libraries(test_sort gtest gmock gtest_main) + add_test(NAME test_sort COMMAND test_sort) +endif() + +if(MINISELECT_ENABLE_FUZZING) + add_subdirectory(benchmark) + include_directories(testing) + add_subdirectory(fuzz) +endif() diff --git a/contrib/miniselect/CONTRIBUTORS b/contrib/miniselect/CONTRIBUTORS new file mode 100644 index 00000000000..75d47387e67 --- /dev/null +++ b/contrib/miniselect/CONTRIBUTORS @@ -0,0 +1 @@ +# contributors (in no particular order) diff --git a/contrib/miniselect/LICENSE_1_0.txt b/contrib/miniselect/LICENSE_1_0.txt new file mode 100644 index 00000000000..36b7cd93cdf --- /dev/null +++ b/contrib/miniselect/LICENSE_1_0.txt @@ -0,0 +1,23 @@ +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. diff --git a/contrib/miniselect/README.md b/contrib/miniselect/README.md new file mode 100644 index 00000000000..cbe576ddba8 --- /dev/null +++ b/contrib/miniselect/README.md @@ -0,0 +1,272 @@ +[![Build Status](https://travis-ci.com/danlark1/miniselect.svg?branch=main)](https://travis-ci.com/danlark1/miniselect) +[![License](https://img.shields.io/badge/License-Boost%201.0-lightblue.svg)](https://www.boost.org/LICENSE_1_0.txt) + +miniselect : Generic selection and partial ordering algorithms +============================================================== + +`miniselect` is a C++ header-only library that contains various generic selection +and partial sorting algorithms with the ease of use, testing, advice on usage and +benchmarking. + +Sorting is everywhere and there are many outstanding sorting algorithms that +compete in speed, comparison count and cache friendliness. However selection +algorithms are always a bit outside of the competition scope, however they are +pretty important, for example, in databases ORDER BY LIMIT N is used extremely +often which can benefit from more optimal selection and partial sorting +algorithms. This library tries to solve this problem with Modern C++. + +* **Easy:** First-class, easy to use dependency and carefully documented APIs and algorithm properties. +* **Fast:** We do care about speed of the algorithms and provide reasonable implementations. +* **Standard compliant:** We provide C++11 compatible APIs that are compliant to the standard [`std::nth_element`](https://en.cppreference.com/w/cpp/algorithm/nth_element) and [`std::partial_sort`](https://en.cppreference.com/w/cpp/algorithm/partial_sort) functions including custom comparators and order guarantees. Just replace the names of the functions in your project and it should work! +* **Well tested:** We test all algorithms with a unified framework, under sanitizers and fuzzing. +* **Benchmarked:** We gather benchmarks for all implementations to better understand good and bad spots. + +Table of Contents +----------------- + +* [Quick Start](#quick-start) +* [Testing](#testing) +* [Documentation](#documentation) +* [Performance results](#performance-results) +* [Real-world usage](#real-world-usage) +* [Contributing](#contributing) +* [Motivation](#motivation) +* [License](#license) + +Quick Start +----------- + +You can either include this project as a cmake dependency and then use the +headers that are provided in the [include](./include) folder or just pass the +[include](./include) folder to your compiler. + +```cpp +#include +#include + +#include "miniselect/median_of_ninthers.h" + +int main() { + std::vector v = {1, 8, 4, 3, 2, 9, 0, 7, 6, 5}; + miniselect::median_of_ninthers_select(v.begin(), v.begin() + 5, v.end()); + for (const int i : v) { + std::cout << i << ' '; + } + return 0; +} +// Compile it `clang++/g++ -I$DIRECTORY/miniselect/include/ example.cpp -std=c++11 -O3 -o example +// Possible output: 0 1 4 3 2 5 8 7 6 9 +``` + +Examples can be found in [examples](./examples). + +We support all compilers starting from GCC 7 and Clang 6. We are also planning +to support Windows, for now it is best effort but no issues are known so far. + +More on which algorithms are available, see [documentation](#documentation). + +Testing +------- + +To test and benchmark, we use [Google benchmark](https://github.com/google/benchmark) library. +Simply do in the root directory: + +```console +# Check out the library. +$ git clone https://github.com/google/benchmark.git +# Benchmark requires Google Test as a dependency. Add the source tree as a subdirectory. +$ git clone https://github.com/google/googletest.git benchmark/googletest +$ mkdir build && cd build +$ cmake -DMINISELECT_TESTING=on .. +$ make -j +$ ctest -j4 --output-on-failure +``` + +It will create two tests and two benchmarks `test_sort`, `test_select`, +`benchmark_sort`, `benchmark_select`. Use them to validate or contribute. You +can also use `ctest` + +Documentation +------------- + +There are several selection algorithms available, further ![\large n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n) is the number +of elements in the array, ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) is the selection element that is needed to be found (all algorithms are deterministic and not stable unless otherwise is specified): + + +| Name | Average | Best Case | Worst Case | Comparisons | Memory | +|------------------------- |--------------------------------------------------------------------------------------------------------- |--------------------------------------------------------------------------------------------------------- |----------------------------------------------------------------------------------------------------------------------- |---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |--------------------------------------------------------------------------------------------------------------------------------- | +| [pdqselect](./include/miniselect/pdqselect.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+n%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 2.5n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2.5n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | +| [Floyd-Rivest](./include/miniselect/floyd_rivest_select.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | Avg: ![\large n + \min(k, n - k) + O(\sqrt{n \log n})](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n+%2B+%5Cmin%28k%2C+n+-+k%29+%2B+O%28%5Csqrt%7Bn+%5Clog+n%7D%29) | ![\large O(\log \log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+%5Clog+n%29) | +| [Median Of Medians](./include/miniselect/median_of_medians.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | Between ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) and ![\large 22n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+22n). Random data ![\large 2.5n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2.5n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | +| [Median Of Ninthers](./include/miniselect/median_of_ninthers.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | Between ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) and ![\large 12n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+12.5n). Random data ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | +| [Median Of 3 Random](./include/miniselect/median_of_3_random.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | +| [libstdc++ (introselect)](https://github.com/gcc-mirror/gcc/blob/e0af865ab9d9d5b6b3ac7fdde26cf9bbf635b6b4/libstdc%2B%2B-v3/include/bits/stl_algo.h#L4748) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+n%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | +| [libc++ (median of 3)](https://github.com/llvm/llvm-project/blob/3ed89b51da38f081fedb57727076262abb81d149/libcxx/include/algorithm#L5159) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | + +For sorting the situation is similar except every line adds ![\large O(k\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28k%5Clog+k%29) comparisons and pdqselect is using ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) memory with one more general exception called partial sorting in C++ standard library. + +| Name | Average | Best Case | Worst Case | Comparisons | Memory | +|-------------------|-----------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------- | +| [std::partial_sort](https://github.com/llvm/llvm-project/blob/3ed89b51da38f081fedb57727076262abb81d149/libcxx/include/algorithm#L5074) | ![\large O(n\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+k%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+k%29) | ![\large n\log k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n%5Clog+k) on average, for some data patterns might be better | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | + +## API + +All functions end either in `select`, either in `partial_sort` and +their behavior is exactly the same as for +[`std::nth_element`](https://en.cppreference.com/w/cpp/algorithm/nth_element) +and [`std::partial_sort`](https://en.cppreference.com/w/cpp/algorithm/partial_sort) +respectively, i.e. they accept 3 arguments as `first`, `middle`, `end` iterators +and an optional comparator. Several notes: + +* You should not throw exceptions from `Compare` function. Standard library +also does not specify the behavior in that matter. +* We don't support ParallelSTL for now. +* C++20 constexpr specifiers might be added but currently we don't have them +because of some floating point math in several algorithms. +* All functions are in the `miniselect` namespace. See the example for that. + +- pdqselect + - This algorithm is based on [`pdqsort`](https://github.com/orlp/pdqsort) which is acknowledged as one of the fastest generic sort algorithms. + - **Location:** [`miniselect/pdqselect.h`](./include/miniselect/pdqselect.h). + - **Functions:** `pdqselect`, `pdqselect_branchless`, `pdqpartial_sort`, `pdqpartial_sort_branchless`. Branchless version uses branchless partition algorithm provided by [`pdqsort`](https://github.com/orlp/pdqsort). Use it if your comparison function is branchless, it might give performance for very big ranges. + - **Performance advice:** Use it when you need to sort a big chunk so that ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) is close to ![\large n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n). + +

+ +- Floyd-Rivest + - This algorithm is based on [Floyd-Rivest algorithm](https://en.wikipedia.org/wiki/Floyd%E2%80%93Rivest_algorithm). + - **Location:** [`miniselect/floyd_rivest_select.h`](./include/miniselect/floyd_rivest_select.h). + - **Functions:** `floyd_rivest_select`, `floyd_rivest_partial_sort`. + - **Performance advice:** Given that this algorithm performs as one of the best on average case in terms of comparisons and speed, we highly advise to + at least try this in your project. Especially it is good for small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) or types that are expensive to compare (for example, strings). But even for median the benchmarks show it outperforms others. It is not easy for this algorithm to build a reasonable worst case but one of examples when this algorithm does not perform well is when there are lots of similar values of linear size (random01 dataset showed some moderate penalties). + +We present here two gifs, for median and for ![\large k = n / 10](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k+%3D+n+%2F+10) order statistic. + +

+ + +

+ +- Median Of Medians + - This algorithm is based on [Median of Medians](https://en.wikipedia.org/wiki/Median_of_medians) algorithm, one of the first deterministic linear time worst case median algorithm + - **Location:** [`miniselect/median_of_medians.h`](./include/miniselect/median_of_medians.h). + - **Functions:** `median_of_medians_select`, `median_of_medians_partial_sort`. + - **Performance advice:** This algorithm does not show advantages over others, implemented for historical reasons and for bechmarking. + +

+ +- Median Of Ninthers + - This algorithm is based on [Fast Deterministic Selection](https://erdani.com/research/sea2017.pdf) paper by Andrei Alexandrescu, one of the latest and fastest deterministic linear time worst case median algorithms + - **Location:** [`miniselect/median_of_ninthers.h`](./include/miniselect/median_of_ninthers.h). + - **Functions:** `median_of_ninthers_select`, `median_of_ninthers_partial_sort`. + - **Performance advice:** Use this algorithm if you absolutely need linear time worst case scenario for selection algorithm. This algorithm shows some strengths over other deterministic [`PICK`](https://en.wikipedia.org/wiki/Median_of_medians) algorithms and has lower constanst than MedianOfMedians. + +

+ +- Median Of 3 Random + - This algorithm is based on QuickSelect with the random median of 3 pivot choice algorithm (it chooses random 3 elements in the range and takes the middle value). It is a rando + - **Location:** [`miniselect/median_of_3_random.h`](./include/miniselect/median_of_3_random.h). + - **Functions:** `median_of_3_random_select`, `median_of_3_random_partial_sort`. + - **Performance advice:** This is a randomized algorithm and also it did not show any strengths against Median Of Ninthers. + +

+ +- Introselect + - This algorithm is based on [Introselect](https://en.wikipedia.org/wiki/Introselect) algorithm, it is used in libstdc++ in `std::nth_element`, however instead of falling back to MedianOfMedians it is using HeapSelect which adds logarithm to its worst complexity. + - **Location:** ``. + - **Functions:** `std::nth_element`. + - **Performance advice:** This algorithm is used in standard library and is not recommended to use if you are looking for performance. + +

+ +- Median Of 3 + - This algorithm is based on QuickSelect with median of 3 pivot choice algorithm (the middle value between begin, mid and end values), it is used in libc++ in `std::nth_element`. + - **Location:** ``. + - **Functions:** `std::nth_element`. + - **Performance advice:** This algorithm is used in standard library and is not recommended to use if you are looking for performance. + +

+ +- `std::partial_sort` + - This algorithm has [heap-based solutions](https://en.wikipedia.org/wiki/Partial_sorting) both in libc++ and libstdc++, from the first ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) elements the max heap is built, then one by one the elements are trying to be pushed to that heap with HeapSort in the end. + - **Location:** ``. + - **Functions:** `std::partial_sort`. + - **Performance advice:** This algorithm is very good for random data and small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) and might outperform all selection+sort algorithms. However, for descending data it starts to significantly degrade and is not recommended for use if you have such patterns in real data. + +

+ +## Other algorithms to come + +* Kiwiel modification of FloydRivest algorithm which is described in [On Floyd and Rivest’s SELECT algorithm](https://core.ac.uk/download/pdf/82672439.pdf) with ternary and quintary pivots. +* Combination of FloydRivest and pdqsort pivot strategies, currently all experiments did not show any boost. + +Performance results +------------------- + +We use 10 datasets and 8 algorithms with 10000000 elements to find median and +other ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) on `Intel(R) Core(TM) i5-4200H CPU @ 2.80GHz` for `std::vector`, +for median the benchmarks are the following: + +![median](benches/plots/result_10000000_5000000.png) + +![median](benches/plots/result_comparisons_10000000_5000000.png) + +For smaller ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k), +for example, 1000, the results are the following + +![k equals 1000](benches/plots/result_10000000_1000.png) + +![k equals 1000](benches/plots/result_comparisons_10000000_1000.png) + +Other benchmarks can be found [here](https://drive.google.com/drive/folders/1DHEaeXgZuX6AJ9eByeZ8iQVQv0ueP8XM). + +The benchmarks for number of swaps will be later. + +Real-world usage +---------------- + +- [Yandex ClickHouse](https://github.com/yandex/ClickHouse) + +If you are planning to use miniselect in your product, please work from one of +our releases and if you wish, you can write the acknowledgment in this section +for visibility. + +Contributing +------------ + +Patches are welcome with new algorithms! You should add the selection algorithm +together with the partial sorting algorithm in [include](./include), add +tests in [testing](./testing) and ideally run benchmarks to see how it performs. +If you also have some data cases to test against, we would be more than happy +to merge them. + +Motivation +---------- + +Firstly the author was interested if any research had been done for small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) +in selection algorithms and was struggling to find working implementations to +compare different approaches from standard library and quickselect algorithms. +After that it turned out that the problem is much more interesting than it looks +like and after reading The Art of Computer Programming from Donald Knuth about +minimum comparison sorting and selection algorithms the author decided to look +through all non-popular algorithms and try them out. + +The author have not found any decent library for selection algorithms and little +research is published in open source, so that they decided to merge all that +implementations and compare them with possible merging of different ideas +into a decent one algorithm for most needs. For a big story of adventures see +the author's blog post TODO. + +License +------- + +The code is made available under the [Boost License 1.0](https://boost.org/LICENSE_1_0.txt). + +Third-Party Libraries Used and Adjusted +--------------------------------------- + +| Library | License | +|---------------------|--------------------------------------------------------------------------------------------------| +| pdqsort | [MIT](https://github.com/orlp/pdqsort/blob/47a46767d76fc852284eaa083e4b7034ee6e2559/license.txt) | +| MedianOfNinthers | [Boost License 1.0](https://github.com/andralex/MedianOfNinthers/blob/master/LICENSE_1_0.txt) | + diff --git a/contrib/miniselect/benches/bench_common.h b/contrib/miniselect/benches/bench_common.h new file mode 100644 index 00000000000..b49b55dac9d --- /dev/null +++ b/contrib/miniselect/benches/bench_common.h @@ -0,0 +1,170 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include +#include +#include + +namespace miniselect { +namespace datagens { + +struct Random { + static std::vector Gen(size_t size) { + std::random_device rnd_device; + std::mt19937_64 mersenne_engine{rnd_device()}; + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size; ++i) { + v.push_back(i); + } + std::shuffle(v.begin(), v.end(), mersenne_engine); + return v; + } +}; + +struct Shuffled16 { + static std::vector Gen(size_t size) { + std::random_device rnd_device; + std::mt19937_64 mersenne_engine{rnd_device()}; + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size; ++i) { + v.push_back(i % 16); + } + std::shuffle(v.begin(), v.end(), mersenne_engine); + return v; + } +}; + +struct Random01 { + static std::vector Gen(size_t size) { + std::random_device rnd_device; + std::mt19937_64 mersenne_engine{rnd_device()}; + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size; ++i) { + v.push_back(i % 2); + } + std::shuffle(v.begin(), v.end(), mersenne_engine); + return v; + } +}; + +struct Ascending { + static std::vector Gen(size_t size) { + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size; ++i) { + v.push_back(i); + } + return v; + } +}; + +struct Descending { + static std::vector Gen(size_t size) { + std::vector v; + v.reserve(size); + for (int i = size - 1; i >= 0; --i) { + v.push_back(i); + } + return v; + } +}; + +struct PipeOrgan { + static std::vector Gen(size_t size) { + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size / 2; ++i) { + v.push_back(i); + } + for (size_t i = size / 2; i < size; ++i) { + v.push_back(size - i); + } + return v; + } +}; + +struct PushFront { + static std::vector Gen(size_t size) { + std::vector v; + v.reserve(size); + for (size_t i = 1; i < size; ++i) { + v.push_back(i); + } + v.push_back(0); + return v; + } +}; + +struct PushMiddle { + static std::vector Gen(size_t size) { + std::vector v; + v.reserve(size); + for (size_t i = 0; i < size; ++i) { + if (i != size / 2) { + v.push_back(i); + } + } + v.push_back(size / 2); + return v; + } +}; + +struct Median3Killer { + static std::vector Gen(size_t size) { + size_t k = size / 2; + std::vector v; + v.reserve(size); + for (size_t i = 1; i < k + 1; ++i) { + if (i & 1) { + v.push_back(i); + } else { + v.push_back(k + i - 1); + } + } + for (size_t i = 1; i < k + 1; ++i) { + v.push_back(2 * i); + } + return v; + } +}; + +#define BENCH_IMPL(BENCH, GEN, IMPL) \ + BENCHMARK_TEMPLATE(BENCH, GEN, IMPL) \ + ->Unit(benchmark::kMicrosecond) \ + ->Arg(kSize - 10) \ + ->Arg(kSize / 2) \ + ->Arg(10000) \ + ->Arg(1000) \ + ->Arg(100) \ + ->Arg(10) \ + ->Arg(1) + +#define BENCH_GENS(BENCH, IMPL) \ + BENCH_IMPL(BENCH, datagens::Random, IMPL); \ + BENCH_IMPL(BENCH, datagens::Shuffled16, IMPL); \ + BENCH_IMPL(BENCH, datagens::Random01, IMPL); \ + BENCH_IMPL(BENCH, datagens::Ascending, IMPL); \ + BENCH_IMPL(BENCH, datagens::Descending, IMPL); \ + BENCH_IMPL(BENCH, datagens::PipeOrgan, IMPL); \ + BENCH_IMPL(BENCH, datagens::PushMiddle, IMPL); \ + BENCH_IMPL(BENCH, datagens::PushFront, IMPL); \ + BENCH_IMPL(BENCH, datagens::Median3Killer, IMPL) + +#define BENCH(NAME) \ + BENCH_GENS(NAME, algorithms::FloydRivest); \ + BENCH_GENS(NAME, algorithms::MedianOfNinthers); \ + BENCH_GENS(NAME, algorithms::MedianOfMedians); \ + BENCH_GENS(NAME, algorithms::MedianOf3Random); \ + BENCH_GENS(NAME, algorithms::PDQ); \ + BENCH_GENS(NAME, algorithms::PDQBranchless); \ + BENCH_GENS(NAME, algorithms::STD) + +} // namespace datagens +} // namespace miniselect diff --git a/contrib/miniselect/benches/benchmark_select.cpp b/contrib/miniselect/benches/benchmark_select.cpp new file mode 100644 index 00000000000..2a9b238c90a --- /dev/null +++ b/contrib/miniselect/benches/benchmark_select.cpp @@ -0,0 +1,46 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#include + +#include +#include +#include +#include +#include +#include + +#include "bench_common.h" +#include "test_common.h" + +namespace miniselect { +namespace { + +static constexpr size_t kSize = 65536; + +template +static void BM_sel(benchmark::State& state) { + auto vec = DataGen::Gen(kSize); + const size_t arg = state.range(0); + size_t cnt = 0; + size_t cmp = 0; + for (auto _ : state) { + Impl::Select(vec.begin(), vec.begin() + arg, vec.end(), + [&cmp](const auto& left, const auto& right) { + cmp++; + return left < right; + }); + ++cnt; + benchmark::DoNotOptimize(vec[arg]); + } + state.counters["Comparisons"] = 1.0 * cmp / cnt; +} + +BENCH(BM_sel); + +} // namespace +} // namespace miniselect + +BENCHMARK_MAIN(); diff --git a/contrib/miniselect/benches/benchmark_sort.cpp b/contrib/miniselect/benches/benchmark_sort.cpp new file mode 100644 index 00000000000..8b3bbd1a77f --- /dev/null +++ b/contrib/miniselect/benches/benchmark_sort.cpp @@ -0,0 +1,46 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#include + +#include +#include +#include +#include +#include +#include + +#include "bench_common.h" +#include "test_common.h" + +namespace miniselect { +namespace { + +static constexpr size_t kSize = 65536; + +template +static void BM_sort(benchmark::State& state) { + auto vec = DataGen::Gen(kSize); + const size_t arg = state.range(0); + size_t cnt = 0; + size_t cmp = 0; + for (auto _ : state) { + Impl::Sort(vec.begin(), vec.begin() + arg, vec.end(), + [&cmp](const auto& left, const auto& right) { + cmp++; + return left < right; + }); + ++cnt; + benchmark::DoNotOptimize(vec[arg]); + } + state.counters["Comparisons"] = 1.0 * cmp / cnt; +} + +BENCH(BM_sort); + +} // namespace +} // namespace miniselect + +BENCHMARK_MAIN(); diff --git a/contrib/miniselect/examples/example.cpp b/contrib/miniselect/examples/example.cpp new file mode 100644 index 00000000000..183e81ae1b6 --- /dev/null +++ b/contrib/miniselect/examples/example.cpp @@ -0,0 +1,18 @@ +#include +#include + +#include "miniselect/median_of_ninthers.h" + +int main() { + std::vector v = {1, 8, 4, 3, 2, 9, 0, 7, 6, 5}; + miniselect::median_of_ninthers_select(v.begin(), v.begin() + 5, v.end()); + for (const int i : v) { + std::cout << i << ' '; + } + return 0; +} + +// Compile it `clang++/g++ -I$DIRECTORY/miniselect/include/ example.cpp -std=c++11 -O3 -o example + +// Possible output: 0 1 4 3 2 5 8 7 6 9 +// ^ on the right place diff --git a/contrib/miniselect/fuzz/CMakeLists.txt b/contrib/miniselect/fuzz/CMakeLists.txt new file mode 100644 index 00000000000..38473bd78ad --- /dev/null +++ b/contrib/miniselect/fuzz/CMakeLists.txt @@ -0,0 +1,38 @@ +cmake_minimum_required(VERSION 3.7) + +project(fuzz) + +option(ENABLE_FUZZING "enable building the fuzzers" ON) +set(CMAKE_CXX_STANDARD 17) + +if(ENABLE_FUZZING) + set(MINISELECT_FUZZ_LDFLAGS "" CACHE STRING "LDFLAGS for the fuzz targets") + + add_library(miniselect-fuzzer INTERFACE) + target_link_libraries(miniselect-fuzzer INTERFACE gtest) + target_link_libraries(miniselect-fuzzer INTERFACE ${MINISELECT_FUZZ_LDFLAGS}) + + if(MINISELECT_FUZZ_LINKMAIN) + target_sources(simdjson-fuzzer INTERFACE $/main.cpp) + endif() + + # Define the fuzzers + add_custom_target(all_fuzzers) + + set(fuzzernames) + function(implement_fuzzer name) + add_executable(${name} ${name}.cpp) + target_link_libraries(${name} PRIVATE miniselect-fuzzer) + add_dependencies(all_fuzzers ${name}) + set(fuzzernames ${fuzzernames} ${name} PARENT_SCOPE) + endfunction() + + implement_fuzzer(fuzz_select) + implement_fuzzer(fuzz_string_select) + implement_fuzzer(fuzz_sort) + implement_fuzzer(fuzz_string_sort) + + # to be able to get a list of all fuzzers from within a script + add_custom_target(print_all_fuzzernames + COMMAND ${CMAKE_COMMAND} -E echo ${fuzzernames}) +endif() diff --git a/contrib/miniselect/fuzz/build_like_oss_fuzz.sh b/contrib/miniselect/fuzz/build_like_oss_fuzz.sh new file mode 100755 index 00000000000..547348133a9 --- /dev/null +++ b/contrib/miniselect/fuzz/build_like_oss_fuzz.sh @@ -0,0 +1,22 @@ +#!/bin/sh +# +# This script emulates how oss fuzz invokes the build +# process, handy for trouble shooting cmake issues and possibly +# recreating testcases. For proper debugging of the oss fuzz +# build, follow the procedure at https://google.github.io/oss-fuzz/getting-started/new-project-guide/#testing-locally + +set -eu + +ossfuzz=$(readlink -f $(dirname $0))/ossfuzz.sh + +mkdir -p ossfuzz-out +export OUT=$(pwd)/ossfuzz-out +export CC=clang +export CXX="clang++" +export CFLAGS="-fsanitize=fuzzer-no-link" +export CXXFLAGS="-fsanitize=fuzzer-no-link,address,undefined -O1" +export LIB_FUZZING_ENGINE="-fsanitize=fuzzer" + +$ossfuzz + +echo "look at the results in $OUT" diff --git a/contrib/miniselect/fuzz/fuzz_select.cpp b/contrib/miniselect/fuzz/fuzz_select.cpp new file mode 100644 index 00000000000..f70980bd0d9 --- /dev/null +++ b/contrib/miniselect/fuzz/fuzz_select.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include + +#include "test_common.h" + +template +void ChooseImplementation(uint8_t byte, std::vector& working, + Iter partition_iter, const ::testing::Types&) { + static_assert(sizeof...(T) < 256); + int i = 0; + constexpr size_t size = sizeof...(T); + ( + [&]() { + if (byte % size == i++) { + T::Select(working.begin(), partition_iter, working.end()); + } + }(), + ...); +} + +// Use the first element as a position into the data +extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, + std::size_t size) { + if (size <= 3) return 0; + uint8_t impl = data[0]; + uint16_t partition_point = 0; + memcpy(&partition_point, data + 1, 2); + partition_point %= (size - 3); + std::vector working(data + 3, data + size); + auto canonical = working; + const auto partition_iter = working.begin() + partition_point; + ChooseImplementation(impl, working, partition_iter, + miniselect::algorithms::All{}); + + if (partition_iter != working.end()) { + const auto& nth = *partition_iter; + bool is_error = false; + if (!std::all_of(working.begin(), partition_iter, + [&](const auto& v) { return v <= nth; })) { + is_error = true; + } + if (!std::all_of(partition_iter, working.end(), + [&](const auto& v) { return v >= nth; })) { + is_error = true; + } + if (is_error) { + std::cerr << "FAILED!\nCanonical: "; + for (const auto& s : canonical) { + std::cerr << static_cast(s) << ' '; + } + std::cerr << std::endl; + std::cerr << "Got: "; + for (const auto& s : working) { + std::cerr << static_cast(s) << ' '; + } + std::cerr << std::endl; + std::cerr << "partition_iter = " << partition_iter - working.begin() + << std::endl; + std::abort(); + } + } + + return 0; +} diff --git a/contrib/miniselect/fuzz/fuzz_sort.cpp b/contrib/miniselect/fuzz/fuzz_sort.cpp new file mode 100644 index 00000000000..ba0a2b6ca3e --- /dev/null +++ b/contrib/miniselect/fuzz/fuzz_sort.cpp @@ -0,0 +1,69 @@ +#include +#include +#include +#include + +#include "test_common.h" + +template +void ChooseImplementation(uint8_t byte, std::vector& working, + Iter partition_iter, const ::testing::Types&) { + static_assert(sizeof...(T) < 256); + int i = 0; + constexpr size_t size = sizeof...(T); + ( + [&]() { + if (byte % size == i++) { + T::Sort(working.begin(), partition_iter, working.end()); + } + }(), + ...); +} + +// Use the first element as a position into the data +extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, + std::size_t size) { + if (size <= 3) return 0; + uint8_t impl = data[0]; + uint16_t partition_point = 0; + memcpy(&partition_point, data + 1, 2); + partition_point %= (size - 3); + std::vector working(data + 3, data + size); + auto canonical = working; + const auto partition_iter = working.begin() + partition_point; + ChooseImplementation(impl, working, partition_iter, + miniselect::algorithms::All{}); + + bool is_error = false; + if (partition_iter != working.end()) { + const auto& nth = *std::min_element(partition_iter, working.end()); + if (!std::all_of(working.begin(), partition_iter, + [&](const auto& v) { return v <= nth; })) { + is_error = true; + } + if (!std::all_of(partition_iter, working.end(), + [&](const auto& v) { return v >= nth; })) { + is_error = true; + } + } + if (!std::is_sorted(working.begin(), partition_iter)) { + is_error = true; + } + if (is_error) { + std::cerr << "FAILED!\nCanonical: "; + for (const auto& s : canonical) { + std::cerr << static_cast(s) << ' '; + } + std::cerr << std::endl; + std::cerr << "Got: "; + for (const auto& s : working) { + std::cerr << static_cast(s) << ' '; + } + std::cerr << std::endl; + std::cerr << "partition_iter = " << partition_iter - working.begin() + << std::endl; + std::abort(); + } + + return 0; +} diff --git a/contrib/miniselect/fuzz/fuzz_string_select.cpp b/contrib/miniselect/fuzz/fuzz_string_select.cpp new file mode 100644 index 00000000000..cd24b376d86 --- /dev/null +++ b/contrib/miniselect/fuzz/fuzz_string_select.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include + +#include "test_common.h" + +template +void ChooseImplementation(uint8_t byte, std::vector& working, + Iter partition_iter, const ::testing::Types&) { + static_assert(sizeof...(T) < 256); + int i = 0; + constexpr size_t size = sizeof...(T); + ( + [&]() { + if (byte % size == i++) { + T::Select(working.begin(), partition_iter, working.end()); + } + }(), + ...); +} + +// Use the first element as a position into the data +extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, + std::size_t size) { + if (size <= 3) return 0; + uint8_t impl = data[0]; + uint16_t partition_point = 0; + memcpy(&partition_point, data + 1, 2); + partition_point %= (size - 3); + std::vector working; + for (auto i = data + 3; i < data + size; ++i) { + std::string s(1, *i); + working.push_back(s); + } + auto canonical = working; + const auto partition_iter = working.begin() + partition_point; + ChooseImplementation(impl, working, partition_iter, + miniselect::algorithms::All{}); + // nth may be the end iterator, in this case nth_element has no effect. + if (partition_iter != working.end()) { + const auto& nth = *partition_iter; + bool is_error = false; + if (!std::all_of(working.begin(), partition_iter, + [&](const auto& v) { return v <= nth; })) { + is_error = true; + } + if (!std::all_of(partition_iter, working.end(), + [&](const auto& v) { return v >= nth; })) { + is_error = true; + } + if (is_error) { + std::cerr << "FAILED!\nCanonical: "; + for (const auto& s : canonical) { + std::cerr << s << ' '; + } + std::cerr << std::endl; + std::cerr << "Got: "; + for (const auto& s : working) { + std::cerr << s << ' '; + } + std::cerr << std::endl; + std::cerr << "partition_iter = " << partition_iter - working.begin() + << std::endl; + std::abort(); + } + } + + return 0; +} diff --git a/contrib/miniselect/fuzz/fuzz_string_sort.cpp b/contrib/miniselect/fuzz/fuzz_string_sort.cpp new file mode 100644 index 00000000000..a797e0d7e22 --- /dev/null +++ b/contrib/miniselect/fuzz/fuzz_string_sort.cpp @@ -0,0 +1,73 @@ +#include +#include +#include +#include + +#include "test_common.h" + +template +void ChooseImplementation(uint8_t byte, std::vector& working, + Iter partition_iter, const ::testing::Types&) { + static_assert(sizeof...(T) < 256); + int i = 0; + constexpr size_t size = sizeof...(T); + ( + [&]() { + if (byte % size == i++) { + T::Sort(working.begin(), partition_iter, working.end()); + } + }(), + ...); +} + +// Use the first element as a position into the data +extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, + std::size_t size) { + if (size <= 3) return 0; + uint8_t impl = data[0]; + uint16_t partition_point = 0; + memcpy(&partition_point, data + 1, 2); + partition_point %= (size - 3); + std::vector working; + for (auto i = data + 3; i < data + size; ++i) { + std::string s(1, *i); + working.push_back(s); + } + auto canonical = working; + const auto partition_iter = working.begin() + partition_point; + ChooseImplementation(impl, working, partition_iter, + miniselect::algorithms::All{}); + // nth may be the end iterator, in this case nth_element has no effect. + bool is_error = false; + if (partition_iter != working.end()) { + const auto& nth = *std::min_element(partition_iter, working.end()); + if (!std::all_of(working.begin(), partition_iter, + [&](const auto& v) { return v <= nth; })) { + is_error = true; + } + if (!std::all_of(partition_iter, working.end(), + [&](const auto& v) { return v >= nth; })) { + is_error = true; + } + } + if (!std::is_sorted(working.begin(), partition_iter)) { + is_error = true; + } + if (is_error) { + std::cerr << "FAILED!\nCanonical: "; + for (const auto& s : canonical) { + std::cerr << s << ' '; + } + std::cerr << std::endl; + std::cerr << "Got: "; + for (const auto& s : working) { + std::cerr << s << ' '; + } + std::cerr << std::endl; + std::cerr << "partition_iter = " << partition_iter - working.begin() + << std::endl; + std::abort(); + } + + return 0; +} diff --git a/contrib/miniselect/fuzz/main.cpp b/contrib/miniselect/fuzz/main.cpp new file mode 100644 index 00000000000..e3377035f33 --- /dev/null +++ b/contrib/miniselect/fuzz/main.cpp @@ -0,0 +1,22 @@ +#include +#include +#include +#include + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t* Data, std::size_t Size); + +int main(int argc, char* argv[]) { + for (int i = 1; i < argc; ++i) { + std::ifstream in(argv[i]); + assert(in); + in.seekg(0, std::ios_base::end); + const auto pos = in.tellg(); + assert(pos >= 0); + in.seekg(0, std::ios_base::beg); + std::vector buf(static_cast(pos)); + in.read(buf.data(), static_cast(buf.size())); + assert(in.gcount() == pos); + LLVMFuzzerTestOneInput(reinterpret_cast(buf.data()), + buf.size()); + } +} diff --git a/contrib/miniselect/fuzz/ossfuzz.sh b/contrib/miniselect/fuzz/ossfuzz.sh new file mode 100755 index 00000000000..83f37f54eee --- /dev/null +++ b/contrib/miniselect/fuzz/ossfuzz.sh @@ -0,0 +1,23 @@ +#!/bin/sh +# +# entry point for oss-fuzz, so that fuzzers +# and build invocation can be changed without having +# to modify the oss-fuzz repo. +# +# invoke it from the git root. + +# make sure to exit on problems +set -eux + +mkdir -p build +cd build + +cmake .. \ +-GNinja \ +-DCMAKE_BUILD_TYPE=Debug \ +-DENABLE_FUZZING=On \ +-DMINISELECT_FUZZ_LINKMAIN=off \ +-DMINISELECT_FUZZ_LDFLAGS=$LIB_FUZZING_ENGINE + +cmake --build . --target all_fuzzers + diff --git a/contrib/miniselect/include/miniselect/floyd_rivest_select.h b/contrib/miniselect/include/miniselect/floyd_rivest_select.h new file mode 100644 index 00000000000..e7d5f80f572 --- /dev/null +++ b/contrib/miniselect/include/miniselect/floyd_rivest_select.h @@ -0,0 +1,120 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace miniselect { +namespace floyd_rivest_detail { + +template +struct CompareRefType { + // Pass the comparator by lvalue reference. Or in debug mode, using a + // debugging wrapper that stores a reference. + using type = typename std::add_lvalue_reference::type; +}; + +template +inline void floyd_rivest_select_loop(Iter begin, Diff left, Diff right, Diff k, + Compare comp) { + while (right > left) { + Diff size = right - left; + if (size > 600) { + Diff n = right - left + 1; + Diff i = k - left + 1; + double z = log(n); + double s = 0.5 * exp(2 * z / 3); + double sd = 0.5 * sqrt(z * s * (n - s) / n); + if (i < n / 2) { + sd *= -1.0; + } + Diff newLeft = std::max(left, (Diff)(k - i * s / n + sd)); + Diff newRight = std::min(right, (Diff)(k + (n - i) * s / n + sd)); + floyd_rivest_select_loop(begin, newLeft, newRight, k, + comp); + } + Diff i = left; + Diff j = right; + std::swap(begin[left], begin[k]); + const bool to_swap = comp(begin[left], begin[right]); + if (to_swap) { + std::swap(begin[left], begin[right]); + } + // Make sure that non copyable types compile. + const auto& t = to_swap ? begin[left] : begin[right]; + while (i < j) { + std::swap(begin[i], begin[j]); + i++; + j--; + while (comp(begin[i], t)) { + i++; + } + while (comp(t, begin[j])) { + j--; + } + } + + if (to_swap) { + std::swap(begin[left], begin[j]); + } else { + j++; + std::swap(begin[right], begin[j]); + } + + if (j <= k) { + left = j + 1; + } + if (k <= j) { + right = j - 1; + } + } +} + +} // namespace floyd_rivest_detail + +template +inline void floyd_rivest_partial_sort(Iter begin, Iter mid, Iter end, + Compare comp) { + if (begin == end) return; + if (begin == mid) return; + using CompType = typename floyd_rivest_detail::CompareRefType::type; + + floyd_rivest_detail::floyd_rivest_select_loop< + Iter, CompType, typename std::iterator_traits::difference_type>( + begin, 0, end - begin - 1, mid - begin - 1, comp); + // std::sort proved to be better than other sorts because of pivoting. + std::sort(begin, mid, comp); +} + +template +inline void floyd_rivest_partial_sort(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + floyd_rivest_partial_sort(begin, mid, end, std::less()); +} + +template +inline void floyd_rivest_select(Iter begin, Iter mid, Iter end, Compare comp) { + if (mid == end) return; + using CompType = typename floyd_rivest_detail::CompareRefType::type; + + floyd_rivest_detail::floyd_rivest_select_loop< + Iter, CompType, typename std::iterator_traits::difference_type>( + begin, 0, end - begin - 1, mid - begin, comp); +} + +template +inline void floyd_rivest_select(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + floyd_rivest_select(begin, mid, end, std::less()); +} + +} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_3_random.h b/contrib/miniselect/include/miniselect/median_of_3_random.h new file mode 100644 index 00000000000..0f7b62fd61c --- /dev/null +++ b/contrib/miniselect/include/miniselect/median_of_3_random.h @@ -0,0 +1,69 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include +#include +#include +#include +#include + +#include "private/median_common.h" + +namespace miniselect { +namespace median_of_3_random_detail { + +template +static inline Iter partition(Iter r, Iter end, Compare&& comp) { + typedef typename std::iterator_traits::difference_type T; + const T len = end - r; + assert(len >= 3); + static std::mt19937_64 gen(1); + std::uniform_int_distribution dis(0, len - 1); + T x = dis(gen); + T y = dis(gen); + T z = dis(gen); + return median_common_detail::pivotPartition( + r, median_common_detail::medianIndex(r, x, y, z, comp), len, comp); +} + +} // namespace median_of_3_random_detail + +template +inline void median_of_3_random_select(Iter begin, Iter mid, Iter end, + Compare comp) { + if (mid == end) return; + using CompType = typename floyd_rivest_detail::CompareRefType::type; + + median_common_detail::quickselect< + Iter, CompType, &median_of_3_random_detail::partition>( + begin, mid, end, comp); +} + +template +inline void median_of_3_random_select(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_3_random_select(begin, mid, end, std::less()); +} + +template +inline void median_of_3_random_sort(Iter begin, Iter mid, Iter end, + Compare comp) { + if (begin == mid) return; + using CompType = typename floyd_rivest_detail::CompareRefType::type; + median_common_detail::quickselect< + Iter, CompType, &median_of_3_random_detail::partition>( + begin, mid - 1, end, comp); + std::sort(begin, mid, comp); +} + +template +inline void median_of_3_random_sort(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_3_random_sort(begin, mid, end, std::less()); +} + +} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_medians.h b/contrib/miniselect/include/miniselect/median_of_medians.h new file mode 100644 index 00000000000..922401b12d0 --- /dev/null +++ b/contrib/miniselect/include/miniselect/median_of_medians.h @@ -0,0 +1,71 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include +#include +#include +#include + +#include "private/median_common.h" + +namespace miniselect { +namespace median_of_medians_detail { + +template +static inline Iter partition(Iter r, Iter end, Compare&& comp) { + using CompType = typename median_common_detail::CompareRefType::type; + const size_t len = end - r; + if (len < 5) { + return median_common_detail::pivotPartition(r, len / 2, len, comp); + } + size_t j = 0; + for (size_t i = 4; i < len; i += 5, ++j) { + median_common_detail::partition5(r, i - 4, i - 3, i, i - 2, i - 1, comp); + std::swap(r[i], r[j]); + } + median_common_detail::quickselect(r, r + j / 2, + r + j, comp); + return median_common_detail::pivotPartition(r, j / 2, len, comp); +} + +} // namespace median_of_medians_detail + +template +inline void median_of_medians_select(Iter begin, Iter mid, Iter end, + Compare comp) { + if (mid == end) return; + using CompType = typename median_common_detail::CompareRefType::type; + + median_common_detail::quickselect< + Iter, CompType, &median_of_medians_detail::partition>( + begin, mid, end, comp); +} + +template +inline void median_of_medians_select(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_medians_select(begin, mid, end, std::less()); +} + +template +inline void median_of_medians_sort(Iter begin, Iter mid, Iter end, + Compare comp) { + if (begin == mid) return; + using CompType = typename median_common_detail::CompareRefType::type; + median_common_detail::quickselect< + Iter, CompType, &median_of_medians_detail::partition>( + begin, mid - 1, end, comp); + std::sort(begin, mid, comp); +} + +template +inline void median_of_medians_sort(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_medians_sort(begin, mid, end, std::less()); +} + +} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_ninthers.h b/contrib/miniselect/include/miniselect/median_of_ninthers.h new file mode 100644 index 00000000000..099786cf518 --- /dev/null +++ b/contrib/miniselect/include/miniselect/median_of_ninthers.h @@ -0,0 +1,190 @@ +/* Copyright Andrei Alexandrescu, 2016-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +// Adjusted from Alexandrescu paper to support arbitrary comparators. +#pragma once + +#include +#include +#include +#include +#include + +#include "private/median_common.h" + +namespace miniselect { +namespace median_of_ninthers_detail { + +template +void adaptiveQuickselect(Iter r, size_t n, size_t length, Compare&& comp); + +/** +Median of minima +*/ +template +size_t medianOfMinima(Iter const r, const size_t n, const size_t length, + Compare&& comp) { + assert(length >= 2); + assert(n * 4 <= length); + assert(n > 0); + const size_t subset = n * 2, computeMinOver = (length - subset) / subset; + assert(computeMinOver > 0); + for (size_t i = 0, j = subset; i < subset; ++i) { + const auto limit = j + computeMinOver; + size_t minIndex = j; + while (++j < limit) + if (comp(r[j], r[minIndex])) minIndex = j; + if (comp(r[minIndex], r[i])) std::swap(r[i], r[minIndex]); + assert(j < length || i + 1 == subset); + } + adaptiveQuickselect(r, n, subset, comp); + return median_common_detail::expandPartition(r, 0, n, subset, length, comp); +} + +/** +Median of maxima +*/ +template +size_t medianOfMaxima(Iter const r, const size_t n, const size_t length, + Compare&& comp) { + assert(length >= 2); + assert(n * 4 >= length * 3 && n < length); + const size_t subset = (length - n) * 2, subsetStart = length - subset, + computeMaxOver = subsetStart / subset; + assert(computeMaxOver > 0); + for (size_t i = subsetStart, j = i - subset * computeMaxOver; i < length; + ++i) { + const auto limit = j + computeMaxOver; + size_t maxIndex = j; + while (++j < limit) + if (comp(r[maxIndex], r[j])) maxIndex = j; + if (comp(r[i], r[maxIndex])) std::swap(r[i], r[maxIndex]); + assert(j != 0 || i + 1 == length); + } + adaptiveQuickselect(r + subsetStart, length - n, subset, comp); + return median_common_detail::expandPartition(r, subsetStart, n, length, + length, comp); +} + +/** +Partitions r[0 .. length] using a pivot of its own choosing. Attempts to pick a +pivot that approximates the median. Returns the position of the pivot. +*/ +template +size_t medianOfNinthers(Iter const r, const size_t length, Compare&& comp) { + assert(length >= 12); + const auto frac = length <= 1024 + ? length / 12 + : length <= 128 * 1024 ? length / 64 : length / 1024; + auto pivot = frac / 2; + const auto lo = length / 2 - pivot, hi = lo + frac; + assert(lo >= frac * 4); + assert(length - hi >= frac * 4); + assert(lo / 2 >= pivot); + const auto gap = (length - 9 * frac) / 4; + auto a = lo - 4 * frac - gap, b = hi + gap; + for (size_t i = lo; i < hi; ++i, a += 3, b += 3) { + median_common_detail::ninther(r, a, i - frac, b, a + 1, i, b + 1, a + 2, + i + frac, b + 2, comp); + } + + adaptiveQuickselect(r + lo, pivot, frac, comp); + return median_common_detail::expandPartition(r, lo, lo + pivot, hi, length, + comp); +} + +/** +Quickselect driver for medianOfNinthers, medianOfMinima, and medianOfMaxima. +Dispathes to each depending on the relationship between n (the sought order +statistics) and length. +*/ +template +void adaptiveQuickselect(Iter r, size_t n, size_t length, Compare&& comp) { + assert(n < length); + for (;;) { + // Decide strategy for partitioning + if (n == 0) { + // That would be the max + auto pivot = n; + for (++n; n < length; ++n) + if (comp(r[n], r[pivot])) pivot = n; + std::swap(r[0], r[pivot]); + return; + } + if (n + 1 == length) { + // That would be the min + auto pivot = 0; + for (n = 1; n < length; ++n) + if (comp(r[pivot], r[n])) pivot = n; + std::swap(r[pivot], r[length - 1]); + return; + } + assert(n < length); + size_t pivot; + if (length <= 16) + pivot = median_common_detail::pivotPartition(r, n, length, comp) - r; + else if (n * 6 <= length) + pivot = medianOfMinima(r, n, length, comp); + else if (n * 6 >= length * 5) + pivot = medianOfMaxima(r, n, length, comp); + else + pivot = medianOfNinthers(r, length, comp); + + // See how the pivot fares + if (pivot == n) { + return; + } + if (pivot > n) { + length = pivot; + } else { + ++pivot; + r += pivot; + length -= pivot; + n -= pivot; + } + } +} + +} // namespace median_of_ninthers_detail + +template +inline void median_of_ninthers_select(Iter begin, Iter mid, Iter end, + Compare comp) { + if (mid == end) return; + using CompType = typename median_common_detail::CompareRefType::type; + + median_of_ninthers_detail::adaptiveQuickselect( + begin, mid - begin, end - begin, comp); +} + +template +inline void median_of_ninthers_select(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_ninthers_select(begin, mid, end, std::less()); +} + +template +inline void median_of_ninthers_sort(Iter begin, Iter mid, Iter end, + Compare comp) { + if (begin == mid) return; + using CompType = typename median_common_detail::CompareRefType::type; + + median_of_ninthers_detail::adaptiveQuickselect( + begin, mid - begin - 1, end - begin, comp); + std::sort(begin, mid, comp); +} + +template +inline void median_of_ninthers_sort(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + median_of_ninthers_sort(begin, mid, end, std::less()); +} + +} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/pdqselect.h b/contrib/miniselect/include/miniselect/pdqselect.h new file mode 100644 index 00000000000..0a22d059103 --- /dev/null +++ b/contrib/miniselect/include/miniselect/pdqselect.h @@ -0,0 +1,935 @@ +/* + pdqsort.h - Pattern-defeating quicksort. + + Copyright (c) 2015 Orson Peters + + This software is provided 'as-is', without any express or implied warranty. + In no event will the authors be held liable for any damages arising from the + use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software in a + product, an acknowledgment in the product documentation would be appreciated + but is not required. + + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + + 3. This notice may not be removed or altered from any source distribution. +*/ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +// Adjusted by Danila Kutenin to support pdqselect and pdqpartial_sort. + +#ifndef PDQSORT_H +#define PDQSORT_H + +#include +#include +#include +#include +#include + +#if __cplusplus >= 201103L +#include +#include +#define PDQSORT_PREFER_MOVE(x) std::move(x) +#else +#define PDQSORT_PREFER_MOVE(x) (x) +#endif + +namespace miniselect { +namespace pdqsort_detail { + +template +struct CompareRefType { + // Pass the comparator by lvalue reference. Or in debug mode, using a + // debugging wrapper that stores a reference. + using type = typename std::add_lvalue_reference::type; +}; + +enum { + // Partitions below this size are sorted using insertion sort. + insertion_sort_threshold = 24, + + // Partitions above this size use Tukey's ninther to select the pivot. + ninther_threshold = 128, + + // When we detect an already sorted partition, attempt an insertion sort that + // allows this + // amount of element moves before giving up. + partial_insertion_sort_limit = 8, + + // Must be multiple of 8 due to loop unrolling, and < 256 to fit in unsigned + // char. + block_size = 64, + + // Cacheline size, assumes power of two. + cacheline_size = 64 + +}; + +#if __cplusplus >= 201103L +template +struct is_default_compare : std::false_type {}; +template +struct is_default_compare> : std::true_type {}; +template +struct is_default_compare> : std::true_type {}; +#endif + +// Returns floor(log2(n)), assumes n > 0. +template +inline int log2(T n) { + int log = 0; + while (n >>= 1) ++log; + return log; +} + +// Sorts [begin, end) using insertion sort with the given comparison function. +template +inline void insertion_sort(Iter begin, Iter end, Compare& comp) { + typedef typename std::iterator_traits::value_type T; + if (begin == end) return; + + for (Iter cur = begin + 1; cur != end; ++cur) { + Iter sift = cur; + Iter sift_1 = cur - 1; + + // Compare first so we can avoid 2 moves for an element already positioned + // correctly. + if (comp(*sift, *sift_1)) { + T tmp = PDQSORT_PREFER_MOVE(*sift); + + do { + *sift-- = PDQSORT_PREFER_MOVE(*sift_1); + } while (sift != begin && comp(tmp, *--sift_1)); + + *sift = PDQSORT_PREFER_MOVE(tmp); + } + } +} + +// Sorts [begin, end) using insertion sort with the given comparison function. +// Assumes +// *(begin - 1) is an element smaller than or equal to any element in [begin, +// end). +template +inline void unguarded_insertion_sort(Iter begin, Iter end, Compare& comp) { + typedef typename std::iterator_traits::value_type T; + if (begin == end) return; + + for (Iter cur = begin + 1; cur != end; ++cur) { + Iter sift = cur; + Iter sift_1 = cur - 1; + + // Compare first so we can avoid 2 moves for an element already positioned + // correctly. + if (comp(*sift, *sift_1)) { + T tmp = PDQSORT_PREFER_MOVE(*sift); + + do { + *sift-- = PDQSORT_PREFER_MOVE(*sift_1); + } while (comp(tmp, *--sift_1)); + + *sift = PDQSORT_PREFER_MOVE(tmp); + } + } +} + +// Attempts to use insertion sort on [begin, end). Will return false if more +// than partial_insertion_sort_limit elements were moved, and abort sorting. +// Otherwise it will successfully sort and return true. +template +inline bool partial_insertion_sort(Iter begin, Iter end, Compare& comp) { + typedef typename std::iterator_traits::value_type T; + if (begin == end) return true; + + std::size_t limit = 0; + for (Iter cur = begin + 1; cur != end; ++cur) { + Iter sift = cur; + Iter sift_1 = cur - 1; + + // Compare first so we can avoid 2 moves for an element already positioned + // correctly. + if (comp(*sift, *sift_1)) { + T tmp = PDQSORT_PREFER_MOVE(*sift); + + do { + *sift-- = PDQSORT_PREFER_MOVE(*sift_1); + } while (sift != begin && comp(tmp, *--sift_1)); + + *sift = PDQSORT_PREFER_MOVE(tmp); + limit += cur - sift; + } + + if (limit > partial_insertion_sort_limit) return false; + } + + return true; +} + +template +inline void sort2(Iter a, Iter b, Compare& comp) { + if (comp(*b, *a)) std::iter_swap(a, b); +} + +// Sorts the elements *a, *b and *c using comparison function comp. +template +inline void sort3(Iter a, Iter b, Iter c, Compare& comp) { + sort2(a, b, comp); + sort2(b, c, comp); + sort2(a, b, comp); +} + +template +inline T* align_cacheline(T* p) { +#if defined(UINTPTR_MAX) && __cplusplus >= 201103L + std::uintptr_t ip = reinterpret_cast(p); +#else + std::size_t ip = reinterpret_cast(p); +#endif + ip = (ip + cacheline_size - 1) & -cacheline_size; + return reinterpret_cast(ip); +} + +template +inline void swap_offsets(Iter first, Iter last, unsigned char* offsets_l, + unsigned char* offsets_r, int num, bool use_swaps) { + typedef typename std::iterator_traits::value_type T; + if (use_swaps) { + // This case is needed for the descending distribution, where we need + // to have proper swapping for pdqsort to remain O(n). + for (int i = 0; i < num; ++i) { + std::iter_swap(first + offsets_l[i], last - offsets_r[i]); + } + } else if (num > 0) { + Iter l = first + offsets_l[0]; + Iter r = last - offsets_r[0]; + T tmp(PDQSORT_PREFER_MOVE(*l)); + *l = PDQSORT_PREFER_MOVE(*r); + for (int i = 1; i < num; ++i) { + l = first + offsets_l[i]; + *r = PDQSORT_PREFER_MOVE(*l); + r = last - offsets_r[i]; + *l = PDQSORT_PREFER_MOVE(*r); + } + *r = PDQSORT_PREFER_MOVE(tmp); + } +} + +// Partitions [begin, end) around pivot *begin using comparison function comp. +// Elements equal to the pivot are put in the right-hand partition. Returns the +// position of the pivot after partitioning and whether the passed sequence +// already was correctly partitioned. Assumes the pivot is a median of at least +// 3 elements and that [begin, end) is at least insertion_sort_threshold long. +// Uses branchless partitioning. +template +inline std::pair partition_right_branchless(Iter begin, Iter end, + Compare& comp) { + typedef typename std::iterator_traits::value_type T; + + // Move pivot into local for speed. + T pivot(PDQSORT_PREFER_MOVE(*begin)); + Iter first = begin; + Iter last = end; + + // Find the first element greater than or equal than the pivot (the median of + // 3 guarantees this exists). + while (comp(*++first, pivot)) + ; + + // Find the first element strictly smaller than the pivot. We have to guard + // this search if there was no element before *first. + if (first - 1 == begin) + while (first < last && !comp(*--last, pivot)) + ; + else + while (!comp(*--last, pivot)) + ; + + // If the first pair of elements that should be swapped to partition are the + // same element, the passed in sequence already was correctly partitioned. + bool already_partitioned = first >= last; + if (!already_partitioned) { + std::iter_swap(first, last); + ++first; + } + + // The following branchless partitioning is derived from "BlockQuicksort: How + // Branch Mispredictions don’t affect Quicksort" by Stefan Edelkamp and Armin + // Weiss. + unsigned char offsets_l_storage[block_size + cacheline_size]; + unsigned char offsets_r_storage[block_size + cacheline_size]; + unsigned char* offsets_l = align_cacheline(offsets_l_storage); + unsigned char* offsets_r = align_cacheline(offsets_r_storage); + int num_l, num_r, start_l, start_r; + num_l = num_r = start_l = start_r = 0; + + while (last - first > 2 * block_size) { + // Fill up offset blocks with elements that are on the wrong side. + if (num_l == 0) { + start_l = 0; + Iter it = first; + for (unsigned char i = 0; i < block_size;) { + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + } + } + if (num_r == 0) { + start_r = 0; + Iter it = last; + for (unsigned char i = 0; i < block_size;) { + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + } + } + + // Swap elements and update block sizes and first/last boundaries. + int num = std::min(num_l, num_r); + swap_offsets(first, last, offsets_l + start_l, offsets_r + start_r, num, + num_l == num_r); + num_l -= num; + num_r -= num; + start_l += num; + start_r += num; + if (num_l == 0) first += block_size; + if (num_r == 0) last -= block_size; + } + + int l_size = 0, r_size = 0; + int unknown_left = (int)(last - first) - ((num_r || num_l) ? block_size : 0); + if (num_r) { + // Handle leftover block by assigning the unknown elements to the other + // block. + l_size = unknown_left; + r_size = block_size; + } else if (num_l) { + l_size = block_size; + r_size = unknown_left; + } else { + // No leftover block, split the unknown elements in two blocks. + l_size = unknown_left / 2; + r_size = unknown_left - l_size; + } + + // Fill offset buffers if needed. + if (unknown_left && !num_l) { + start_l = 0; + Iter it = first; + for (unsigned char i = 0; i < l_size;) { + offsets_l[num_l] = i++; + num_l += !comp(*it, pivot); + ++it; + } + } + if (unknown_left && !num_r) { + start_r = 0; + Iter it = last; + for (unsigned char i = 0; i < r_size;) { + offsets_r[num_r] = ++i; + num_r += comp(*--it, pivot); + } + } + + int num = std::min(num_l, num_r); + swap_offsets(first, last, offsets_l + start_l, offsets_r + start_r, num, + num_l == num_r); + num_l -= num; + num_r -= num; + start_l += num; + start_r += num; + if (num_l == 0) first += l_size; + if (num_r == 0) last -= r_size; + + // We have now fully identified [first, last)'s proper position. Swap the last + // elements. + if (num_l) { + offsets_l += start_l; + while (num_l--) std::iter_swap(first + offsets_l[num_l], --last); + first = last; + } + if (num_r) { + offsets_r += start_r; + while (num_r--) std::iter_swap(last - offsets_r[num_r], first), ++first; + last = first; + } + + // Put the pivot in the right place. + Iter pivot_pos = first - 1; + *begin = PDQSORT_PREFER_MOVE(*pivot_pos); + *pivot_pos = PDQSORT_PREFER_MOVE(pivot); + + return std::make_pair(pivot_pos, already_partitioned); +} + +// Partitions [begin, end) around pivot *begin using comparison function comp. +// Elements equal to the pivot are put in the right-hand partition. Returns the +// position of the pivot after partitioning and whether the passed sequence +// already was correctly partitioned. Assumes the pivot is a median of at least +// 3 elements and that [begin, end) is at least insertion_sort_threshold long. +template +inline std::pair partition_right(Iter begin, Iter end, + Compare& comp) { + typedef typename std::iterator_traits::value_type T; + + // Move pivot into local for speed. + T pivot(PDQSORT_PREFER_MOVE(*begin)); + + Iter first = begin; + Iter last = end; + + // Find the first element greater than or equal than the pivot (the median of + // 3 guarantees this exists). + while (comp(*++first, pivot)) + ; + + // Find the first element strictly smaller than the pivot. We have to guard + // this search if there was no element before *first. + if (first - 1 == begin) + while (first < last && !comp(*--last, pivot)) + ; + else + while (!comp(*--last, pivot)) + ; + + // If the first pair of elements that should be swapped to partition are the + // same element, the passed in sequence already was correctly partitioned. + bool already_partitioned = first >= last; + + // Keep swapping pairs of elements that are on the wrong side of the pivot. + // Previously swapped pairs guard the searches, which is why the first + // iteration is special-cased above. + while (first < last) { + std::iter_swap(first, last); + while (comp(*++first, pivot)) + ; + while (!comp(*--last, pivot)) + ; + } + + // Put the pivot in the right place. + Iter pivot_pos = first - 1; + *begin = PDQSORT_PREFER_MOVE(*pivot_pos); + *pivot_pos = PDQSORT_PREFER_MOVE(pivot); + + return std::make_pair(pivot_pos, already_partitioned); +} + +// Similar function to the one above, except elements equal to the pivot are put +// to the left of the pivot and it doesn't check or return if the passed +// sequence already was partitioned. Since this is rarely used (the many equal +// case), and in that case pdqsort already has O(n) performance, no block +// quicksort is applied here for simplicity. +template +inline Iter partition_left(Iter begin, Iter end, Compare& comp) { + typedef typename std::iterator_traits::value_type T; + + T pivot(PDQSORT_PREFER_MOVE(*begin)); + Iter first = begin; + Iter last = end; + + while (comp(pivot, *--last)) + ; + + if (last + 1 == end) + while (first < last && !comp(pivot, *++first)) + ; + else + while (!comp(pivot, *++first)) + ; + + while (first < last) { + std::iter_swap(first, last); + while (comp(pivot, *--last)) + ; + while (!comp(pivot, *++first)) + ; + } + + Iter pivot_pos = last; + *begin = PDQSORT_PREFER_MOVE(*pivot_pos); + *pivot_pos = PDQSORT_PREFER_MOVE(pivot); + + return pivot_pos; +} + +template +inline void pdqsort_loop(Iter begin, Iter end, Compare& comp, int bad_allowed, + bool leftmost = true) { + typedef typename std::iterator_traits::difference_type diff_t; + + // Use a while loop for tail recursion elimination. + while (true) { + diff_t size = end - begin; + + // Insertion sort is faster for small arrays. + if (size < insertion_sort_threshold) { + if (leftmost) + insertion_sort(begin, end, comp); + else + unguarded_insertion_sort(begin, end, comp); + return; + } + + // Choose pivot as median of 3 or pseudomedian of 9. + diff_t s2 = size / 2; + if (size > ninther_threshold) { + sort3(begin, begin + s2, end - 1, comp); + sort3(begin + 1, begin + (s2 - 1), end - 2, comp); + sort3(begin + 2, begin + (s2 + 1), end - 3, comp); + sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); + std::iter_swap(begin, begin + s2); + } else + sort3(begin + s2, begin, end - 1, comp); + + // If *(begin - 1) is the end of the right partition of a previous partition + // operation there is no element in [begin, end) that is smaller than + // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change + // strategy, putting equal elements in the left partition, greater elements + // in the right partition. We do not have to recurse on the left partition, + // since it's sorted (all equal). + if (!leftmost && !comp(*(begin - 1), *begin)) { + begin = partition_left(begin, end, comp) + 1; + continue; + } + + // Partition and get results. + std::pair part_result = + Branchless ? partition_right_branchless(begin, end, comp) + : partition_right(begin, end, comp); + Iter pivot_pos = part_result.first; + bool already_partitioned = part_result.second; + + // Check for a highly unbalanced partition. + diff_t l_size = pivot_pos - begin; + diff_t r_size = end - (pivot_pos + 1); + bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; + + // If we got a highly unbalanced partition we shuffle elements to break many + // patterns. + if (highly_unbalanced) { + // If we had too many bad partitions, switch to heapsort to guarantee O(n + // log n). + if (--bad_allowed == 0) { + std::make_heap(begin, end, comp); + std::sort_heap(begin, end, comp); + return; + } + + if (l_size >= insertion_sort_threshold) { + std::iter_swap(begin, begin + l_size / 4); + std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); + + if (l_size > ninther_threshold) { + std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); + std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); + std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); + std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); + } + } + + if (r_size >= insertion_sort_threshold) { + std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); + std::iter_swap(end - 1, end - r_size / 4); + + if (r_size > ninther_threshold) { + std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); + std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); + std::iter_swap(end - 2, end - (1 + r_size / 4)); + std::iter_swap(end - 3, end - (2 + r_size / 4)); + } + } + } else { + // If we were decently balanced and we tried to sort an already + // partitioned sequence try to use insertion sort. + if (already_partitioned && + partial_insertion_sort(begin, pivot_pos, comp) && + partial_insertion_sort(pivot_pos + 1, end, comp)) + return; + } + + // Sort the left partition first using recursion and do tail recursion + // elimination for the right-hand partition. + pdqsort_loop(begin, pivot_pos, comp, bad_allowed, + leftmost); + begin = pivot_pos + 1; + leftmost = false; + } +} + +template +inline void pdqpartial_sort_loop(Iter begin, Iter mid, Iter end, Compare& comp, + int bad_allowed, bool leftmost = true) { + typedef typename std::iterator_traits::difference_type diff_t; + + // Use a while loop for tail recursion elimination. + while (true) { + diff_t size = end - begin; + + // Insertion sort is faster for small arrays. + if (size < insertion_sort_threshold) { + if (leftmost) + insertion_sort(begin, end, comp); + else + unguarded_insertion_sort(begin, end, comp); + return; + } + + // Choose pivot as median of 3 or pseudomedian of 9. + diff_t s2 = size / 2; + if (size > ninther_threshold) { + sort3(begin, begin + s2, end - 1, comp); + sort3(begin + 1, begin + (s2 - 1), end - 2, comp); + sort3(begin + 2, begin + (s2 + 1), end - 3, comp); + sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); + std::iter_swap(begin, begin + s2); + } else + sort3(begin + s2, begin, end - 1, comp); + + // If *(begin - 1) is the end of the right partition of a previous partition + // operation there is no element in [begin, end) that is smaller than + // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change + // strategy, putting equal elements in the left partition, greater elements + // in the right partition. We do not have to recurse on the left partition, + // since it's sorted (all equal). + if (!leftmost && !comp(*(begin - 1), *begin)) { + begin = partition_left(begin, end, comp) + 1; + continue; + } + + // Partition and get results. + std::pair part_result = + Branchless ? partition_right_branchless(begin, end, comp) + : partition_right(begin, end, comp); + Iter pivot_pos = part_result.first; + bool already_partitioned = part_result.second; + + // Check for a highly unbalanced partition. + diff_t l_size = pivot_pos - begin; + diff_t r_size = end - (pivot_pos + 1); + bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; + + // If we got a highly unbalanced partition we shuffle elements to break many + // patterns. + if (highly_unbalanced) { + // If we had too many bad partitions, switch to heapsort to guarantee O(n + // log n). + if (--bad_allowed == 0) { + std::make_heap(begin, end, comp); + std::sort_heap(begin, end, comp); + return; + } + + if (l_size >= insertion_sort_threshold) { + std::iter_swap(begin, begin + l_size / 4); + std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); + + if (l_size > ninther_threshold) { + std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); + std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); + std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); + std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); + } + } + + if (r_size >= insertion_sort_threshold) { + std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); + std::iter_swap(end - 1, end - r_size / 4); + + if (r_size > ninther_threshold) { + std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); + std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); + std::iter_swap(end - 2, end - (1 + r_size / 4)); + std::iter_swap(end - 3, end - (2 + r_size / 4)); + } + } + } else { + // If we were decently balanced and we tried to sort an already + // partitioned sequence try to use insertion sort. + if (already_partitioned && + partial_insertion_sort(begin, pivot_pos, comp) && + partial_insertion_sort(pivot_pos + 1, end, comp)) + return; + } + + // Sort the left partition first using recursion and do tail recursion + // elimination for the right-hand partition. + if (pivot_pos < mid) { + pdqsort_loop(begin, pivot_pos, comp, + bad_allowed, leftmost); + begin = pivot_pos + 1; + leftmost = false; + } else { + end = pivot_pos; + } + } +} + +template +inline void pdqselect_loop(Iter begin, Iter mid, Iter end, Compare& comp, + int bad_allowed, bool leftmost = true) { + typedef typename std::iterator_traits::difference_type diff_t; + + // Use a while loop for tail recursion elimination. + while (true) { + diff_t size = end - begin; + + // Insertion sort is faster for small arrays. + if (size < insertion_sort_threshold) { + if (leftmost) + insertion_sort(begin, end, comp); + else + unguarded_insertion_sort(begin, end, comp); + return; + } + + // Choose pivot as median of 3 or pseudomedian of 9. + diff_t s2 = size / 2; + if (size > ninther_threshold) { + sort3(begin, begin + s2, end - 1, comp); + sort3(begin + 1, begin + (s2 - 1), end - 2, comp); + sort3(begin + 2, begin + (s2 + 1), end - 3, comp); + sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); + std::iter_swap(begin, begin + s2); + } else + sort3(begin + s2, begin, end - 1, comp); + + // If *(begin - 1) is the end of the right partition of a previous partition + // operation there is no element in [begin, end) that is smaller than + // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change + // strategy, putting equal elements in the left partition, greater elements + // in the right partition. We do not have to recurse on the left partition, + // since it's sorted (all equal). + if (!leftmost && !comp(*(begin - 1), *begin)) { + begin = partition_left(begin, end, comp) + 1; + continue; + } + + // Partition and get results. + std::pair part_result = + Branchless ? partition_right_branchless(begin, end, comp) + : partition_right(begin, end, comp); + Iter pivot_pos = part_result.first; + bool already_partitioned = part_result.second; + + // Check for a highly unbalanced partition. + diff_t l_size = pivot_pos - begin; + diff_t r_size = end - (pivot_pos + 1); + bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; + + // If we got a highly unbalanced partition we shuffle elements to break many + // patterns. + if (highly_unbalanced) { + // If we had too many bad partitions, switch to heapsort to guarantee O(n + // log n). + if (--bad_allowed == 0) { + std::nth_element(begin, mid, end, comp); + return; + } + + if (l_size >= insertion_sort_threshold) { + std::iter_swap(begin, begin + l_size / 4); + std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); + + if (l_size > ninther_threshold) { + std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); + std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); + std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); + std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); + } + } + + if (r_size >= insertion_sort_threshold) { + std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); + std::iter_swap(end - 1, end - r_size / 4); + + if (r_size > ninther_threshold) { + std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); + std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); + std::iter_swap(end - 2, end - (1 + r_size / 4)); + std::iter_swap(end - 3, end - (2 + r_size / 4)); + } + } + } else { + // If we were decently balanced and we tried to sort an already + // partitioned sequence try to use insertion sort. + if (already_partitioned && + partial_insertion_sort(begin, pivot_pos, comp) && + partial_insertion_sort(pivot_pos + 1, end, comp)) + return; + } + // Sort the left partition first using recursion and do tail recursion + // elimination for the right-hand partition. + if (pivot_pos < mid) { + begin = pivot_pos + 1; + leftmost = false; + } else { + end = pivot_pos; + } + } +} +} // namespace pdqsort_detail + +template +inline void pdqsort(Iter begin, Iter end, Compare comp) { + if (begin == end) return; + +#if __cplusplus >= 201103L + pdqsort_detail::pdqsort_loop< + Iter, Compare, + pdqsort_detail::is_default_compare< + typename std::decay::type>::value && + std::is_arithmetic< + typename std::iterator_traits::value_type>::value>( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#else + pdqsort_detail::pdqsort_loop( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#endif +} + +template +inline void pdqsort(Iter begin, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqsort(begin, end, std::less()); +} + +template +inline void pdqsort_branchless(Iter begin, Iter end, Compare comp) { + if (begin == end) return; + pdqsort_detail::pdqsort_loop( + begin, end, comp, pdqsort_detail::log2(end - begin)); +} + +template +inline void pdqsort_branchless(Iter begin, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqsort_branchless(begin, end, std::less()); +} + +template +inline void pdqpartial_sort(Iter begin, Iter mid, Iter end, Compare comp) { + if (begin == end) return; + +#if __cplusplus >= 201103L + pdqsort_detail::pdqpartial_sort_loop< + Iter, Compare, + pdqsort_detail::is_default_compare< + typename std::decay::type>::value && + std::is_arithmetic< + typename std::iterator_traits::value_type>::value>( + begin, mid, end, comp, pdqsort_detail::log2(end - begin)); +#else + pdqsort_detail::pdqpartial_sort_loop( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#endif +} + +template +inline void pdqpartial_sort(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqpartial_sort(begin, mid, end, std::less()); +} + +template +inline void pdqpartial_sort_branchless(Iter begin, Iter mid, Iter end, + Compare comp) { + if (begin == end) return; + pdqsort_detail::pdqpartial_sort_loop( + begin, mid, end, comp, pdqsort_detail::log2(end - begin)); +} + +template +inline void pdqpartial_sort_branchless(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqpartial_sort_branchless(begin, mid, end, std::less()); +} + +template +inline void pdqselect(Iter begin, Iter mid, Iter end, Compare comp) { + if (mid == end) return; + using CompType = typename median_common_detail::CompareRefType::type; + +#if __cplusplus >= 201103L + pdqsort_detail::pdqselect_loop< + Iter, CompType, + pdqsort_detail::is_default_compare< + typename std::decay::type>::value && + std::is_arithmetic< + typename std::iterator_traits::value_type>::value>( + begin, mid, end, comp, pdqsort_detail::log2(end - begin)); +#else + pdqsort_detail::pdqselect_loop( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#endif +} + +template +inline void pdqselect(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqselect(begin, mid, end, std::less()); +} + +template +inline void pdqselect_branchless(Iter begin, Iter mid, Iter end, Compare comp) { + if (mid == end) return; + using CompType = typename median_common_detail::CompareRefType::type; + pdqsort_detail::pdqselect_loop( + begin, mid, end, comp, pdqsort_detail::log2(end - begin)); +} + +template +inline void pdqselect_branchless(Iter begin, Iter mid, Iter end) { + typedef typename std::iterator_traits::value_type T; + pdqselect_branchless(begin, mid, end, std::less()); +} + +#undef PDQSORT_PREFER_MOVE + +#endif + +} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/private/median_common.h b/contrib/miniselect/include/miniselect/private/median_common.h new file mode 100644 index 00000000000..30cb1323bbf --- /dev/null +++ b/contrib/miniselect/include/miniselect/private/median_common.h @@ -0,0 +1,437 @@ +/* Copyright Andrei Alexandrescu, 2016-, + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include +#include +#include + +namespace miniselect { +namespace median_common_detail { + +template +struct CompareRefType { + // Pass the comparator by lvalue reference. Or in debug mode, using a + // debugging wrapper that stores a reference. + using type = typename std::add_lvalue_reference::type; +}; +/** +Swaps the median of r[a], r[b], and r[c] into r[b]. +*/ +template +void median3(Iter r, size_t a, size_t b, size_t c, Compare&& comp) { + if (comp(r[b], r[a])) // b < a + { + if (comp(r[b], r[c])) // b < a, b < c + { + if (comp(r[c], r[a])) // b < c < a + std::swap(r[b], r[c]); + else // b < a <= c + std::swap(r[b], r[a]); + } + } else if (comp(r[c], r[b])) // a <= b, c < b + { + if (comp(r[c], r[a])) // c < a <= b + std::swap(r[b], r[a]); + else // a <= c < b + std::swap(r[b], r[c]); + } +} + +/** +Sorts in place r[a], r[b], and r[c]. +*/ +template +void sort3(Iter r, size_t a, size_t b, size_t c, Compare&& comp) { + typedef typename std::iterator_traits::value_type T; + if (comp(r[b], r[a])) // b < a + { + if (comp(r[c], r[b])) // c < b < a + { + std::swap(r[a], r[c]); // a < b < c + } else // b < a, b <= c + { + T t = std::move(r[a]); + r[a] = std::move(r[b]); + if (comp(r[c], t)) // b <= c < a + { + r[b] = std::move(r[c]); + r[c] = std::move(t); + } else // b < a <= c + { + r[b] = std::move(t); + } + } + } else if (comp(r[c], r[b])) // a <= b, c < b + { + T t = std::move(r[c]); + r[c] = std::move(r[b]); + if (comp(t, r[a])) // c < a < b + { + r[b] = std::move(r[a]); + r[a] = std::move(t); + } else // a <= c < b + { + r[b] = std::move(t); + } + } + + assert(!comp(r[b], r[a]) && !comp(r[c], r[b])); +} + +/** +If leanRight == false, swaps the lower median of r[a]...r[d] into r[b] and +the minimum into r[a]. If leanRight == true, swaps the upper median of +r[a]...r[d] into r[c] and the minimum into r[d]. +*/ +template +void partition4(Iter r, size_t a, size_t b, size_t c, size_t d, + Compare&& comp) { + assert(a != b && a != c && a != d && b != c && b != d && c != d); + /* static */ if (leanRight) { + // In the median of 5 algorithm, consider r[e] infinite + if (comp(r[c], r[a])) { + std::swap(r[a], r[c]); + } // a <= c + if (comp(r[d], r[b])) { + std::swap(r[b], r[d]); + } // a <= c, b <= d + if (comp(r[d], r[c])) { + std::swap(r[c], r[d]); // a <= d, b <= c < d + std::swap(r[a], r[b]); // b <= d, a <= c < d + } // a <= c <= d, b <= d + if (comp(r[c], r[b])) { // a <= c <= d, c < b <= d + std::swap(r[b], r[c]); // a <= b <= c <= d + } // a <= b <= c <= d + } else { + // In the median of 5 algorithm consider r[a] infinitely small, then + // change b->a. c->b, d->c, e->d + if (comp(r[c], r[a])) { + std::swap(r[a], r[c]); + } + if (comp(r[c], r[b])) { + std::swap(r[b], r[c]); + } + if (comp(r[d], r[a])) { + std::swap(r[a], r[d]); + } + if (comp(r[d], r[b])) { + std::swap(r[b], r[d]); + } else { + if (comp(r[b], r[a])) { + std::swap(r[a], r[b]); + } + } + } +} + +/** +Places the median of r[a]...r[e] in r[c] and partitions the other elements +around it. +*/ +template +void partition5(Iter r, size_t a, size_t b, size_t c, size_t d, size_t e, + Compare&& comp) { + assert(a != b && a != c && a != d && a != e && b != c && b != d && b != e && + c != d && c != e && d != e); + if (comp(r[c], r[a])) { + std::swap(r[a], r[c]); + } + if (comp(r[d], r[b])) { + std::swap(r[b], r[d]); + } + if (comp(r[d], r[c])) { + std::swap(r[c], r[d]); + std::swap(r[a], r[b]); + } + if (comp(r[e], r[b])) { + std::swap(r[b], r[e]); + } + if (comp(r[e], r[c])) { + std::swap(r[c], r[e]); + if (comp(r[c], r[a])) { + std::swap(r[a], r[c]); + } + } else { + if (comp(r[c], r[b])) { + std::swap(r[b], r[c]); + } + } +} + +/** +Implements Hoare partition. +*/ +template +Iter pivotPartition(Iter r, size_t k, size_t length, Compare&& comp) { + assert(k < length); + std::swap(*r, r[k]); + size_t lo = 1, hi = length - 1; + for (;; ++lo, --hi) { + for (;; ++lo) { + if (lo > hi) goto loop_done; + if (!comp(r[lo], *r)) break; + } + // found the left bound: r[lo] >= r[0] + assert(lo <= hi); + for (; comp(*r, r[hi]); --hi) { + } + if (lo >= hi) break; + // found the right bound: r[hi] <= r[0], swap & make progress + std::swap(r[lo], r[hi]); + } +loop_done: + --lo; + std::swap(r[lo], *r); + return r + lo; +} + +/** +Implements the quickselect algorithm, parameterized with a partition function. +*/ +template +void quickselect(Iter r, Iter mid, Iter end, Compare&& comp) { + if (r == end || mid >= end) return; + assert(r <= mid && mid < end); + for (;;) switch (end - r) { + case 1: + return; + case 2: + if (comp(r[1], *r)) std::swap(*r, r[1]); + return; + case 3: + sort3(r, 0, 1, 2, comp); + return; + case 4: + switch (mid - r) { + case 0: + goto select_min; + case 1: + partition4(r, 0, 1, 2, 3, comp); + break; + case 2: + partition4(r, 0, 1, 2, 3, comp); + break; + case 3: + goto select_max; + default: + assert(false); + } + return; + default: + assert(end - r > 4); + if (r == mid) { + select_min: + auto pivot = r; + for (++mid; mid < end; ++mid) + if (comp(*mid, *pivot)) pivot = mid; + std::swap(*r, *pivot); + return; + } + if (mid + 1 == end) { + select_max: + auto pivot = r; + for (mid = r + 1; mid < end; ++mid) + if (comp(*pivot, *mid)) pivot = mid; + std::swap(*pivot, end[-1]); + return; + } + auto pivot = partition(r, end, comp); + if (pivot == mid) return; + if (mid < pivot) { + end = pivot; + } else { + r = pivot + 1; + } + } +} + +/** +Returns the index of the median of r[a], r[b], and r[c] without writing +anything. +*/ +template +size_t medianIndex(const Iter r, size_t a, size_t b, size_t c, Compare&& comp) { + if (r[a] > r[c]) std::swap(a, c); + if (r[b] > r[c]) return c; + if (comp(r[b], r[a])) return a; + return b; +} + +/** +Returns the index of the median of r[a], r[b], r[c], and r[d] without writing +anything. If leanRight is true, computes the upper median. Otherwise, conputes +the lower median. +*/ +template +static size_t medianIndex(Iter r, size_t a, size_t b, size_t c, size_t d, + Compare&& comp) { + if (comp(r[d], r[c])) std::swap(c, d); + assert(r[c] <= r[d]); + /* static */ if (leanRight) { + if (comp(r[c], r[a])) { + assert(comp(r[c], r[a]) && !comp(r[d], r[c])); // so r[c]) is out + return medianIndex(r, a, b, d, comp); + } + } else { + if (!comp(r[d], r[a])) { + return medianIndex(r, a, b, c, comp); + } + } + // Could return medianIndex(r, b, c, d) but we already know r[c] <= r[d] + if (!comp(r[c], r[b])) return c; + if (comp(r[d], r[b])) return d; + return b; +} + +/** +Tukey's Ninther: compute the median of r[_1], r[_2], r[_3], then the median of +r[_4], r[_5], r[_6], then the median of r[_7], r[_8], r[_9], and then swap the +median of those three medians into r[_5]. +*/ +template +void ninther(Iter r, size_t _1, size_t _2, size_t _3, size_t _4, size_t _5, + size_t _6, size_t _7, size_t _8, size_t _9, Compare&& comp) { + _2 = medianIndex(r, _1, _2, _3, comp); + _8 = medianIndex(r, _7, _8, _9, comp); + if (comp(r[_8], r[_2])) std::swap(_2, _8); + if (comp(r[_6], r[_4])) std::swap(_4, _6); + // Here we know that r[_2] and r[_8] are the other two medians and that + // r[_2] <= r[_8]. We also know that r[_4] <= r[_6] + if (comp(r[_5], r[_4])) { + // r[_4] is the median of r[_4], r[_5], r[_6] + } else if (comp(r[_6], r[_5])) { + // r[_6] is the median of r[_4], r[_5], r[_6] + _4 = _6; + } else { + // Here we know r[_5] is the median of r[_4], r[_5], r[_6] + if (comp(r[_5], r[_2])) return std::swap(r[_5], r[_2]); + if (comp(r[_8], r[_5])) return std::swap(r[_5], r[_8]); + // This is the only path that returns with no swap + return; + } + // Here we know r[_4] is the median of r[_4], r[_5], r[_6] + if (comp(r[_4], r[_2])) + _4 = _2; + else if (comp(r[_8], r[_4])) + _4 = _8; + std::swap(r[_5], r[_4]); +} + +/** +Input assumptions: +(a) hi <= rite +(c) the range r[0 .. hi] contains elements no smaller than r[0] +Output guarantee: same as Hoare partition using r[0] as pivot. Returns the new +position of the pivot. +*/ +template +size_t expandPartitionRight(Iter r, size_t hi, size_t rite, Compare&& comp) { + size_t pivot = 0; + assert(pivot <= hi); + assert(hi <= rite); + // First loop: spend r[pivot .. hi] + for (; pivot < hi; --rite) { + if (rite == hi) goto done; + if (!comp(r[rite], r[0])) continue; + ++pivot; + std::swap(r[rite], r[pivot]); + } + // Second loop: make left and pivot meet + for (; rite > pivot; --rite) { + if (!comp(r[rite], r[0])) continue; + while (rite > pivot) { + ++pivot; + if (comp(r[0], r[pivot])) { + std::swap(r[rite], r[pivot]); + break; + } + } + } + +done: + std::swap(r[0], r[pivot]); + return pivot; +} + +/** +Input assumptions: +(a) lo > 0, lo <= pivot +(b) the range r[lo .. pivot] already contains elements no greater than r[pivot] +Output guarantee: Same as Hoare partition around r[pivot]. Returns the new +position of the pivot. +*/ +template +size_t expandPartitionLeft(Iter r, size_t lo, size_t pivot, Compare&& comp) { + assert(lo > 0 && lo <= pivot); + size_t left = 0; + const auto oldPivot = pivot; + for (; lo < pivot; ++left) { + if (left == lo) goto done; + if (!comp(r[oldPivot], r[left])) continue; + --pivot; + std::swap(r[left], r[pivot]); + } + // Second loop: make left and pivot meet + for (;; ++left) { + if (left == pivot) break; + if (!comp(r[oldPivot], r[left])) continue; + for (;;) { + if (left == pivot) goto done; + --pivot; + if (comp(r[pivot], r[oldPivot])) { + std::swap(r[left], r[pivot]); + break; + } + } + } + +done: + std::swap(r[oldPivot], r[pivot]); + return pivot; +} + +/** +Input assumptions: +(a) lo <= pivot, pivot < hi, hi <= length +(b) the range r[lo .. pivot] already contains elements no greater than +r[pivot] +(c) the range r[pivot .. hi] already contains elements no smaller than +r[pivot] +Output guarantee: Same as Hoare partition around r[pivot], returning the new +position of the pivot. +*/ +template +size_t expandPartition(Iter r, size_t lo, size_t pivot, size_t hi, + size_t length, Compare&& comp) { + assert(lo <= pivot && pivot < hi && hi <= length); + --hi; + --length; + size_t left = 0; + for (;; ++left, --length) { + for (;; ++left) { + if (left == lo) + return pivot + expandPartitionRight(r + pivot, hi - pivot, + length - pivot, comp); + if (comp(r[pivot], r[left])) break; + } + for (;; --length) { + if (length == hi) + return left + + expandPartitionLeft(r + left, lo - left, pivot - left, comp); + if (!comp(r[pivot], r[length])) break; + } + std::swap(r[left], r[length]); + } +} + +} // namespace median_common_detail +} // namespace miniselect diff --git a/contrib/miniselect/testing/test_common.h b/contrib/miniselect/testing/test_common.h new file mode 100644 index 00000000000..df0c179c840 --- /dev/null +++ b/contrib/miniselect/testing/test_common.h @@ -0,0 +1,180 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#pragma once + +#include + +#include + +#include "miniselect/floyd_rivest_select.h" +#include "miniselect/median_of_3_random.h" +#include "miniselect/median_of_medians.h" +#include "miniselect/median_of_ninthers.h" +#include "miniselect/pdqselect.h" + +namespace miniselect { +namespace algorithms { + +struct STD { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + std::partial_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + std::partial_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + std::nth_element(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + std::nth_element(begin, mid, end); + } +}; + +struct PDQ { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + pdqpartial_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + pdqpartial_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + pdqselect(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + pdqselect(begin, mid, end); + } +}; + +struct PDQBranchless { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + pdqpartial_sort_branchless(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + pdqpartial_sort_branchless(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + pdqselect_branchless(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + pdqselect_branchless(begin, mid, end); + } +}; + +struct FloydRivest { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + floyd_rivest_partial_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + floyd_rivest_partial_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + floyd_rivest_select(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + floyd_rivest_select(begin, mid, end); + } +}; + +struct MedianOfNinthers { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_ninthers_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + median_of_ninthers_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_ninthers_select(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + median_of_ninthers_select(begin, mid, end); + } +}; + +struct MedianOfMedians { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_medians_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + median_of_medians_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_medians_select(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + median_of_medians_select(begin, mid, end); + } +}; + +struct MedianOf3Random { + template + static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_3_random_sort(begin, mid, end, std::move(comp)); + } + + template + static void Sort(Iter begin, Iter mid, Iter end) { + median_of_3_random_sort(begin, mid, end); + } + + template + static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { + median_of_3_random_select(begin, mid, end, std::move(comp)); + } + + template + static void Select(Iter begin, Iter mid, Iter end) { + median_of_3_random_select(begin, mid, end); + } +}; + +using All = + ::testing::Types; + +} // namespace algorithms +} // namespace miniselect diff --git a/contrib/miniselect/testing/test_select.cpp b/contrib/miniselect/testing/test_select.cpp new file mode 100644 index 00000000000..9b8e9dce970 --- /dev/null +++ b/contrib/miniselect/testing/test_select.cpp @@ -0,0 +1,231 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#include +#include + +#include +#include +#include +#include +#include + +#include "test_common.h" + +using ::testing::Eq; + +namespace miniselect { +namespace { + +struct IndirectLess { + // Non const comparator with deleted copy. + template + bool operator()(const P &x, const P &y) const { + return *x < *y; + } + IndirectLess(const IndirectLess &) = default; + IndirectLess &operator=(const IndirectLess &) = default; + IndirectLess(IndirectLess &&) = default; + IndirectLess &operator=(IndirectLess &&) = default; +}; + +template +class SelectTest : public ::testing::Test { + public: + using Base = Selector; + + static void TestSelects(size_t N, size_t M) { + ASSERT_NE(N, 0); + ASSERT_GT(N, M); + SCOPED_TRACE(N); + SCOPED_TRACE(M); + std::vector array(N); + for (size_t i = 0; i < N; ++i) { + array[i] = i; + } + auto array_smaller = array; + std::mt19937_64 mersenne_engine; + std::shuffle(array.begin(), array.end(), mersenne_engine); + Selector::Select(array.begin(), array.begin() + M, array.end(), + std::greater()); + EXPECT_EQ(array[M], N - M - 1); + for (size_t i = 0; i < M; ++i) { + EXPECT_GE(array[i], array[M]); + } + for (size_t i = M; i < N; ++i) { + EXPECT_LE(array[i], array[M]); + } + std::shuffle(array_smaller.begin(), array_smaller.end(), mersenne_engine); + Selector::Select(array_smaller.begin(), array_smaller.begin() + M, + array_smaller.end()); + EXPECT_EQ(array_smaller[M], M); + for (size_t i = 0; i < M; ++i) { + EXPECT_LE(array_smaller[i], array_smaller[M]); + } + for (size_t i = M; i < N; ++i) { + EXPECT_GE(array_smaller[i], array_smaller[M]); + } + } + + static void TestSelects(size_t N) { + TestSelects(N, 0); + TestSelects(N, 1); + TestSelects(N, 2); + TestSelects(N, 3); + TestSelects(N, N / 2 - 1); + TestSelects(N, N / 2); + TestSelects(N, N / 2 + 1); + TestSelects(N, N - 2); + TestSelects(N, N - 1); + } + + static void TestManySelects() { + TestSelects(10); + TestSelects(256); + TestSelects(257); + TestSelects(499); + TestSelects(500); + TestSelects(997); + TestSelects(1000); + TestSelects(1000 * 100); + TestSelects(1009); + TestSelects(1009 * 109); + } + + static void TestCustomComparators() { + std::vector> v(1000); + for (int i = 0; static_cast(i) < v.size(); ++i) { + v[i] = std::make_unique(i); + } + Selector::Select(v.begin(), v.begin() + v.size() / 2, v.end(), + IndirectLess{}); + EXPECT_EQ(*v[v.size() / 2], v.size() / 2); + for (size_t i = 0; i < v.size() / 2; ++i) { + ASSERT_NE(v[i], nullptr); + EXPECT_LE(*v[i], v.size() / 2); + } + for (size_t i = v.size() / 2; i < v.size(); ++i) { + ASSERT_NE(v[i], nullptr); + EXPECT_GE(*v[i], v.size() / 2); + } + } + + static void TestRepeat(size_t N, size_t M) { + ASSERT_NE(N, 0); + ASSERT_GT(N, M); + SCOPED_TRACE(N); + SCOPED_TRACE(M); + std::mt19937_64 mersenne_engine(10); + std::vector array(N); + for (size_t i = 0; i < M; ++i) { + array[i] = false; + } + for (size_t i = M; i < N; ++i) { + array[i] = true; + } + std::shuffle(array.begin(), array.end(), mersenne_engine); + Selector::Select(array.begin(), array.begin() + M, array.end()); + EXPECT_EQ(array[M], true); + for (size_t i = 0; i < M; ++i) { + EXPECT_EQ(array[i], false); + } + for (size_t i = M; i < N; ++i) { + EXPECT_EQ(array[i], true); + } + std::shuffle(array.begin(), array.end(), mersenne_engine); + Selector::Select(array.begin(), array.begin() + M / 2, array.end()); + EXPECT_EQ(array[M / 2], false); + for (size_t i = 0; i < M / 2; ++i) { + EXPECT_EQ(array[i], false); + } + std::shuffle(array.begin(), array.end(), mersenne_engine); + Selector::Select(array.begin(), array.begin() + M - 1, array.end()); + EXPECT_EQ(array[M - 1], false); + for (size_t i = 0; i < M - 1; ++i) { + EXPECT_EQ(array[i], false); + } + } + + static void TestRepeats(size_t N) { + TestRepeat(N, 1); + TestRepeat(N, 2); + TestRepeat(N, 3); + TestRepeat(N, N / 2 - 1); + TestRepeat(N, N / 2); + TestRepeat(N, N / 2 + 1); + TestRepeat(N, N - 2); + TestRepeat(N, N - 1); + } + + static void TestManyRepeats() { + TestRepeats(10); + TestRepeats(100); + TestRepeats(257); + TestRepeats(1000); + TestRepeats(100000); + } +}; + +TYPED_TEST_SUITE(SelectTest, algorithms::All); + +TYPED_TEST(SelectTest, TestSmall) { + std::vector v = {"ab", "aaa", "ab"}; + TypeParam::Select(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); + v = {"aba"}; + TypeParam::Select(v.begin(), v.begin(), v.end()); + EXPECT_THAT(v, Eq(std::vector{"aba"})); + v.clear(); + TypeParam::Select(v.begin(), v.end(), v.end()); + EXPECT_TRUE(v.empty()); +} + +TYPED_TEST(SelectTest, TestAnotherSmall) { + std::vector v = {"ab", "ab", "aaa"}; + TypeParam::Select(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); +} + +TYPED_TEST(SelectTest, TestEmptySmall) { + std::vector v = {"", ""}; + TypeParam::Select(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"", ""})); +} + +TYPED_TEST(SelectTest, TestBasic) { TestFixture::TestManySelects(); } + +TYPED_TEST(SelectTest, TestComparators) { + TestFixture::TestCustomComparators(); +} + +TYPED_TEST(SelectTest, TestRepeats) { TestFixture::TestManyRepeats(); } + +TYPED_TEST(SelectTest, TestLast) { + std::vector array(100); + for (size_t i = 0; i < 100; ++i) { + array[i] = i; + } + auto array_smaller = array; + std::mt19937_64 mersenne_engine; + std::shuffle(array.begin(), array.end(), mersenne_engine); + auto copy_array = array; + // Should be no effect. + size_t cmp = 0; + TypeParam::Select(array.begin(), array.end(), array.end(), + [&cmp](const auto &lhs, const auto &rhs) { + ++cmp; + return lhs < rhs; + }); + EXPECT_EQ(cmp, 0); + EXPECT_EQ(copy_array, array); +} + +} // namespace +} // namespace miniselect + +int main(int argc, char **argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/contrib/miniselect/testing/test_sort.cpp b/contrib/miniselect/testing/test_sort.cpp new file mode 100644 index 00000000000..19c6ff036fe --- /dev/null +++ b/contrib/miniselect/testing/test_sort.cpp @@ -0,0 +1,161 @@ +/* Copyright Danila Kutenin, 2020-. + * Distributed under the Boost Software License, Version 1.0. + * (See accompanying file LICENSE_1_0.txt or copy at + * https://boost.org/LICENSE_1_0.txt) + */ +#include +#include + +#include +#include +#include +#include +#include + +#include "test_common.h" + +using ::testing::Eq; + +namespace miniselect { +namespace { + +struct IndirectLess { + // Non const comparator with deleted copy. + template + bool operator()(const P &x, const P &y) const { + return *x < *y; + } + IndirectLess(const IndirectLess &) = default; + IndirectLess &operator=(const IndirectLess &) = default; + IndirectLess(IndirectLess &&) = default; + IndirectLess &operator=(IndirectLess &&) = default; +}; + +template +class PartialSortTest : public ::testing::Test { + public: + static void TestSorts(size_t N, size_t M) { + ASSERT_NE(N, 0); + ASSERT_GE(N, M); + SCOPED_TRACE(N); + SCOPED_TRACE(M); + std::vector array(N); + for (size_t i = 0; i < N; ++i) { + array[i] = i; + } + auto array_smaller = array; + std::mt19937_64 mersenne_engine; + std::shuffle(array.begin(), array.end(), mersenne_engine); + Sorter::Sort(array.begin(), array.begin() + M, array.end(), + std::greater()); + for (size_t i = 0; i < M; ++i) { + EXPECT_EQ(array[i], N - i - 1); + } + std::shuffle(array_smaller.begin(), array_smaller.end(), mersenne_engine); + Sorter::Sort(array_smaller.begin(), array_smaller.begin() + M, + array_smaller.end()); + for (size_t i = 0; i < M; ++i) { + EXPECT_EQ(array_smaller[i], i); + } + } + + static void TestSorts(size_t N) { + TestSorts(N, 0); + TestSorts(N, 1); + TestSorts(N, 2); + TestSorts(N, 3); + TestSorts(N, N / 2 - 1); + TestSorts(N, N / 2); + TestSorts(N, N / 2 + 1); + TestSorts(N, N - 2); + TestSorts(N, N - 1); + TestSorts(N, N); + } + + static void TestManySorts() { + TestSorts(10); + TestSorts(256); + TestSorts(257); + TestSorts(499); + TestSorts(500); + TestSorts(997); + TestSorts(1000); + TestSorts(1000 * 100); + TestSorts(1009); + TestSorts(1009 * 109); + } + + static void TestCustomComparators() { + std::vector> v(1000); + for (int i = 0; static_cast(i) < v.size(); ++i) { + v[i] = std::make_unique(i); + } + Sorter::Sort(v.begin(), v.begin() + v.size() / 2, v.end(), IndirectLess{}); + for (int i = 0; static_cast(i) < v.size() / 2; ++i) { + ASSERT_NE(v[i], nullptr); + EXPECT_EQ(*v[i], i); + } + } +}; + +TYPED_TEST_SUITE(PartialSortTest, algorithms::All); + +TYPED_TEST(PartialSortTest, TestSmall) { + std::vector v = {"ab", "aaa", "ab"}; + TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); + v = {"aba"}; + TypeParam::Sort(v.begin(), v.begin(), v.end()); + EXPECT_THAT(v, Eq(std::vector{"aba"})); + v.clear(); + TypeParam::Sort(v.begin(), v.end(), v.end()); + EXPECT_TRUE(v.empty()); +} + +TYPED_TEST(PartialSortTest, TestAnotherSmall) { + std::vector v = {"ab", "ab", "aaa"}; + TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); +} + +TYPED_TEST(PartialSortTest, TestEmptySmall) { + std::vector v = {"", ""}; + TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); + EXPECT_THAT(v, Eq(std::vector{"", ""})); +} + +TYPED_TEST(PartialSortTest, TestBasic) { TestFixture::TestManySorts(); } + +TYPED_TEST(PartialSortTest, TestComparators) { + TestFixture::TestCustomComparators(); +} + +// The standard says that the order of other elements is unspecified even if +// nothing should be sorted so it fails for libcxx and PDQ which is Ok. Saving +// this test for a reference. +TYPED_TEST(PartialSortTest, DISABLED_TestEmpty) { + std::vector array(100); + for (size_t i = 0; i < 100; ++i) { + array[i] = i; + } + std::mt19937_64 mersenne_engine; + std::shuffle(array.begin(), array.end(), mersenne_engine); + size_t cmp = 0; + auto copy_array = array; + // Should be no effect. + TypeParam::Sort(array.begin(), array.begin(), array.end(), + [&cmp](const auto &lhs, const auto &rhs) { + ++cmp; + return lhs < rhs; + }); + EXPECT_EQ(cmp, 0); + EXPECT_EQ(copy_array, array); +} + +} // namespace +} // namespace miniselect + +int main(int argc, char **argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 085269847e4..7b9c05a1adc 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -321,6 +321,7 @@ target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_CURRENT_BINARY_DI dbms_target_include_directories(PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) +dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) if (ZSTD_LIBRARY) dbms_target_link_libraries(PRIVATE ${ZSTD_LIBRARY}) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 9b948236943..6dbe755f0ba 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -20,6 +20,7 @@ #include #include +#include namespace DB { @@ -782,7 +783,7 @@ void ColumnArray::getPermutationImpl(size_t limit, Permutation & res, Comparator auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); else std::sort(res.begin(), res.end(), less); } @@ -835,7 +836,7 @@ void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRa /// Since then we are working inside the interval. - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index b9549175f6c..b27506c1cfb 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -8,6 +8,7 @@ #include #include +#include #include @@ -162,10 +163,10 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum { const auto& [first, last] = equal_ranges[i]; if (reverse) - std::partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); else - std::partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); auto new_first = first; @@ -193,10 +194,10 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum /// Since then we are working inside the interval. if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); auto new_first = first; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index c33ab34b541..128b595803f 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -253,9 +254,9 @@ protected: sort_end = res.begin() + limit; if (reverse) - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); + miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); else - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); + miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); } }; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 0e44b83791c..41e46a7fa98 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -157,9 +158,9 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir if (limit) { if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); } else { @@ -217,9 +218,9 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu /// Since then we are working inside the interval. if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); auto new_first = first; for (auto j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 3f03734b738..9b87a409aaa 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -393,7 +394,7 @@ void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res /// Since then we are working inside the interval. - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 23798f64a9c..541863486a6 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -313,7 +314,7 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Comparato auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); else std::sort(res.begin(), res.end(), less); } @@ -365,7 +366,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR /// Since then we are working inside the interval. - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index d6e1ca982d6..cbe5c7e11cd 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -9,7 +9,7 @@ #include #include #include - +#include namespace DB { @@ -352,7 +352,7 @@ void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperat if (limit) { - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); } else { diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 733a1510f93..e9af38d6984 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -17,7 +17,7 @@ #include #include #include - +#include #ifdef __SSE2__ #include @@ -156,9 +156,9 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi res[i] = i; if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); + miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); } else { @@ -254,9 +254,9 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire /// Since then, we are working inside the interval. if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) From 8c6471f7c3ac3d0a89b948a2b154aba207992013 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 9 Nov 2020 21:37:05 +0300 Subject: [PATCH 32/54] Retry CI From fdce810237a3cab731ebbd4a4b9af73cfd06f9d7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 22:32:18 +0300 Subject: [PATCH 33/54] Add setting back. --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c8521d2f91b..580756361b1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -399,6 +399,7 @@ class IColumn; \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ + M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \ M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ From f7f7fde3a8d4707accf8477c96c4d2925f33923a Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 9 Nov 2020 22:48:39 +0300 Subject: [PATCH 34/54] Fix fast test to see all others --- .../0_stateless/01525_select_with_offset_fetch_clause.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference index 709116b4746..422a076b0cb 100644 --- a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference @@ -3,7 +3,7 @@ 5 1 1 2 1 -3 4 +3 3 1 1 2 1 3 4 From d0840bfc5db83a8fe3a459c3ea30a7a7b374c116 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 00:53:43 +0300 Subject: [PATCH 35/54] Move miniselect to submodule and replace quantile exact with better algorithm --- .gitmodules | 3 + contrib/miniselect | 1 + contrib/miniselect/.clang-format | 1 - contrib/miniselect/.gitignore | 100 -- contrib/miniselect/.travis.yml | 140 --- contrib/miniselect/AUTHORS | 2 - contrib/miniselect/CMakeLists.txt | 52 - contrib/miniselect/CONTRIBUTORS | 1 - contrib/miniselect/LICENSE_1_0.txt | 23 - contrib/miniselect/README.md | 272 ----- contrib/miniselect/benches/bench_common.h | 170 ---- .../miniselect/benches/benchmark_select.cpp | 46 - contrib/miniselect/benches/benchmark_sort.cpp | 46 - contrib/miniselect/examples/example.cpp | 18 - contrib/miniselect/fuzz/CMakeLists.txt | 38 - .../miniselect/fuzz/build_like_oss_fuzz.sh | 22 - contrib/miniselect/fuzz/fuzz_select.cpp | 66 -- contrib/miniselect/fuzz/fuzz_sort.cpp | 69 -- .../miniselect/fuzz/fuzz_string_select.cpp | 70 -- contrib/miniselect/fuzz/fuzz_string_sort.cpp | 73 -- contrib/miniselect/fuzz/main.cpp | 22 - contrib/miniselect/fuzz/ossfuzz.sh | 23 - .../include/miniselect/floyd_rivest_select.h | 120 --- .../include/miniselect/median_of_3_random.h | 69 -- .../include/miniselect/median_of_medians.h | 71 -- .../include/miniselect/median_of_ninthers.h | 190 ---- .../miniselect/include/miniselect/pdqselect.h | 935 ------------------ .../miniselect/private/median_common.h | 437 -------- contrib/miniselect/testing/test_common.h | 180 ---- contrib/miniselect/testing/test_select.cpp | 231 ----- contrib/miniselect/testing/test_sort.cpp | 161 --- src/AggregateFunctions/QuantileExact.h | 13 +- src/AggregateFunctions/QuantileTiming.h | 5 +- src/Columns/ColumnDecimal.cpp | 4 +- 34 files changed, 16 insertions(+), 3658 deletions(-) create mode 160000 contrib/miniselect delete mode 100644 contrib/miniselect/.clang-format delete mode 100644 contrib/miniselect/.gitignore delete mode 100644 contrib/miniselect/.travis.yml delete mode 100644 contrib/miniselect/AUTHORS delete mode 100644 contrib/miniselect/CMakeLists.txt delete mode 100644 contrib/miniselect/CONTRIBUTORS delete mode 100644 contrib/miniselect/LICENSE_1_0.txt delete mode 100644 contrib/miniselect/README.md delete mode 100644 contrib/miniselect/benches/bench_common.h delete mode 100644 contrib/miniselect/benches/benchmark_select.cpp delete mode 100644 contrib/miniselect/benches/benchmark_sort.cpp delete mode 100644 contrib/miniselect/examples/example.cpp delete mode 100644 contrib/miniselect/fuzz/CMakeLists.txt delete mode 100755 contrib/miniselect/fuzz/build_like_oss_fuzz.sh delete mode 100644 contrib/miniselect/fuzz/fuzz_select.cpp delete mode 100644 contrib/miniselect/fuzz/fuzz_sort.cpp delete mode 100644 contrib/miniselect/fuzz/fuzz_string_select.cpp delete mode 100644 contrib/miniselect/fuzz/fuzz_string_sort.cpp delete mode 100644 contrib/miniselect/fuzz/main.cpp delete mode 100755 contrib/miniselect/fuzz/ossfuzz.sh delete mode 100644 contrib/miniselect/include/miniselect/floyd_rivest_select.h delete mode 100644 contrib/miniselect/include/miniselect/median_of_3_random.h delete mode 100644 contrib/miniselect/include/miniselect/median_of_medians.h delete mode 100644 contrib/miniselect/include/miniselect/median_of_ninthers.h delete mode 100644 contrib/miniselect/include/miniselect/pdqselect.h delete mode 100644 contrib/miniselect/include/miniselect/private/median_common.h delete mode 100644 contrib/miniselect/testing/test_common.h delete mode 100644 contrib/miniselect/testing/test_select.cpp delete mode 100644 contrib/miniselect/testing/test_sort.cpp diff --git a/.gitmodules b/.gitmodules index fdd48fcce01..0e4291eac9e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -190,3 +190,6 @@ path = contrib/croaring url = https://github.com/RoaringBitmap/CRoaring branch = v0.2.66 +[submodule "contrib/miniselect"] + path = contrib/miniselect + url = https://github.com/danlark1/miniselect diff --git a/contrib/miniselect b/contrib/miniselect new file mode 160000 index 00000000000..be0af6bd0b6 --- /dev/null +++ b/contrib/miniselect @@ -0,0 +1 @@ +Subproject commit be0af6bd0b6eb044d1acc4f754b229972d99903a diff --git a/contrib/miniselect/.clang-format b/contrib/miniselect/.clang-format deleted file mode 100644 index f6cb8ad931f..00000000000 --- a/contrib/miniselect/.clang-format +++ /dev/null @@ -1 +0,0 @@ -BasedOnStyle: Google diff --git a/contrib/miniselect/.gitignore b/contrib/miniselect/.gitignore deleted file mode 100644 index f80f36759c8..00000000000 --- a/contrib/miniselect/.gitignore +++ /dev/null @@ -1,100 +0,0 @@ -# eclipse project files -.cproject -.project -.settings - -# emacs temp files -*~ - -# vim temp files -.*.swp - -# XCode -^build/ -*.pbxuser -!default.pbxuser -*.mode1v3 -!default.mode1v3 -*.mode2v3 -!default.mode2v3 -*.perspectivev3 -!default.perspectivev3 -xcuserdata -*.xccheckout -*.moved-aside -DerivedData -*.hmap -*.ipa -*.xcuserstate -*.DS_Store - -# IDE specific folder for JetBrains IDEs -.idea/ -cmake-build-debug/ -cmake-build-release/ - -# Visual Studio Code artifacts -.vscode/* -.history/ - -# Visual Studio artifacts -/VS/ - -# C/C++ build outputs -.build/ -bins -gens -libs -objs - -# C++ ignore from https://github.com/github/gitignore/blob/master/C%2B%2B.gitignore - -# Prerequisites -*.d - -# Compiled Object files -*.slo -*.lo -*.o -*.obj - -# Precompiled Headers -*.gch -*.pch - -# Compiled Dynamic libraries -*.so -*.dylib -*.dll - -# Fortran module files -*.mod -*.smod - -# Compiled Static libraries -*.lai -*.la -*.a -*.lib - -# Executables -*.exe -*.out -*.app - - -# CMake files that may be specific to our installation - -# Build outputs -/build*/ -/visual_studio/ -/benchmark/ - -# Fuzzer outputs generated by instructions in fuzz/Fuzzing.md -/corpus.zip -/ossfuzz-out/ -/out/ - -# Generated docs -/doc/api -*.orig diff --git a/contrib/miniselect/.travis.yml b/contrib/miniselect/.travis.yml deleted file mode 100644 index a5036caf365..00000000000 --- a/contrib/miniselect/.travis.yml +++ /dev/null @@ -1,140 +0,0 @@ -language: cpp - -dist: bionic - -matrix: - include: - - os: linux - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - g++-8 - env: - - COMPILER="CC=gcc-8 && CXX=g++-8" - compiler: gcc-8 - - - os: linux - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - g++-9 - env: - - COMPILER="CC=gcc-9 && CXX=g++-9" - compiler: gcc-9 - - - os: linux - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - g++-10 - env: - - COMPILER="CC=gcc-10 && CXX=g++-10" - compiler: gcc-10 - - - os: linux - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - g++-10 - env: - - COMPILER="CC=gcc-10 && CXX=g++-10" - - SANITIZE="on" - compiler: gcc-10-sanitize - - - os: linux - addons: - apt: - sources: - - llvm-toolchain-bionic-6.0 - packages: - - clang-6.0 - env: - - COMPILER="CC=clang-6.0 && CXX=clang++-6.0" - compiler: clang-6 - - - os: linux - addons: - apt: - sources: - - llvm-toolchain-bionic-7 - packages: - - clang-7 - env: - - COMPILER="CC=clang-7 && CXX=clang++-7" - compiler: clang-7 - - - os: linux - addons: - apt: - sources: - - llvm-toolchain-bionic-8 - packages: - - clang-8 - env: - - COMPILER="CC=clang-8 && CXX=clang++-8" - compiler: clang-8 - - - os: linux - addons: - apt: - sources: - - llvm-toolchain-bionic-9 - packages: - - clang-9 - env: - - COMPILER="CC=clang-9 && CXX=clang++-9" - compiler: clang-9 - - - os: linux - addons: - apt: - packages: - - clang-10 - sources: - - ubuntu-toolchain-r-test - - sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main' - key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' - env: - - COMPILER="CC=clang-10 && CXX=clang++-10" - compiler: clang-10 - - - os: linux - addons: - apt: - packages: - - clang-10 - sources: - - ubuntu-toolchain-r-test - - sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main' - key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' - env: - - COMPILER="CC=clang-10 && CXX=clang++-10" - - SANITIZE="on" - compiler: clang-10-sanitize - -before_install: - - eval "${COMPILER}" - - git clone https://github.com/google/benchmark.git - - git clone https://github.com/google/googletest.git benchmark/googletest - -install: - - export CMAKE_FLAGS="-DMINISELECT_TESTING=on -DCMAKE_BUILD_TYPE=RelWithDebInfo"; - - if [[ "${SANITIZE}" == "on" ]]; then - export CMAKE_FLAGS="${CMAKE_FLAGS} -DMINISELECT_SANITIZE=on"; - fi - - export CTEST_FLAGS="-j4 --output-on-failure -E checkperf" - -script: - - mkdir build - - cd build - - cmake $CMAKE_FLAGS .. - - cmake --build . -- -j2 - - ctest $CTEST_FLAGS diff --git a/contrib/miniselect/AUTHORS b/contrib/miniselect/AUTHORS deleted file mode 100644 index 896a8046a73..00000000000 --- a/contrib/miniselect/AUTHORS +++ /dev/null @@ -1,2 +0,0 @@ -# List of authors for copyright purposes, in no particular order -Danila Kutenin diff --git a/contrib/miniselect/CMakeLists.txt b/contrib/miniselect/CMakeLists.txt deleted file mode 100644 index 09e92031784..00000000000 --- a/contrib/miniselect/CMakeLists.txt +++ /dev/null @@ -1,52 +0,0 @@ -cmake_minimum_required(VERSION 3.7) -project(miniselect) - -option(MINISELECT_TESTING "Building the tests." OFF) -option(MINISELECT_SANITIZE "Building the library with sanitizers." OFF) -option(MINISELECT_BUILD_LIBCXX "Building the library with libcxx." OFF) -option(MINISELECT_ENABLE_FUZZING "Building the library with fuzzing." OFF) - -include_directories(include) - -if (MINISELECT_TESTING) - enable_testing() - set(CMAKE_CXX_STANDARD 17) - if (NOT CMAKE_BUILD_TYPE) - message(STATUS "No build type selected, default to Release") - set(CMAKE_BUILD_TYPE "Release") - endif() - if (MINISELECT_SANITIZE) - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsanitize=address -fno-omit-frame-pointer -fsanitize=undefined -fno-sanitize-recover=all") - endif() - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -g -Wall -Wextra -Wpedantic -Wno-gnu-zero-variadic-macro-arguments") - - if (MINISELECT_BUILD_LIBCXX AND "${CMAKE_CXX_COMPILER_ID}" STREQUAL "Clang") - message(STATUS "Using libcxx as a default standard C++ library") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") - endif() - - add_subdirectory(benchmark) - include_directories(testing) - include_directories(benches) - - add_executable(benchmark_sort benches/benchmark_sort.cpp) - target_link_libraries(benchmark_sort benchmark::benchmark gtest) - add_executable(benchmark_select benches/benchmark_select.cpp) - target_link_libraries(benchmark_select benchmark::benchmark gtest) - - set(TEST_SOURCES testing/test_select.cpp) - add_executable(test_select ${TEST_SOURCES}) - target_link_libraries(test_select gtest gmock gtest_main) - add_test(NAME test_select COMMAND test_select) - - set(TEST_SOURCES testing/test_sort.cpp) - add_executable(test_sort ${TEST_SOURCES}) - target_link_libraries(test_sort gtest gmock gtest_main) - add_test(NAME test_sort COMMAND test_sort) -endif() - -if(MINISELECT_ENABLE_FUZZING) - add_subdirectory(benchmark) - include_directories(testing) - add_subdirectory(fuzz) -endif() diff --git a/contrib/miniselect/CONTRIBUTORS b/contrib/miniselect/CONTRIBUTORS deleted file mode 100644 index 75d47387e67..00000000000 --- a/contrib/miniselect/CONTRIBUTORS +++ /dev/null @@ -1 +0,0 @@ -# contributors (in no particular order) diff --git a/contrib/miniselect/LICENSE_1_0.txt b/contrib/miniselect/LICENSE_1_0.txt deleted file mode 100644 index 36b7cd93cdf..00000000000 --- a/contrib/miniselect/LICENSE_1_0.txt +++ /dev/null @@ -1,23 +0,0 @@ -Boost Software License - Version 1.0 - August 17th, 2003 - -Permission is hereby granted, free of charge, to any person or organization -obtaining a copy of the software and accompanying documentation covered by -this license (the "Software") to use, reproduce, display, distribute, -execute, and transmit the Software, and to prepare derivative works of the -Software, and to permit third-parties to whom the Software is furnished to -do so, all subject to the following: - -The copyright notices in the Software and this entire statement, including -the above license grant, this restriction and the following disclaimer, -must be included in all copies of the Software, in whole or in part, and -all derivative works of the Software, unless such copies or derivative -works are solely in the form of machine-executable object code generated by -a source language processor. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT -SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE -FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, -ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -DEALINGS IN THE SOFTWARE. diff --git a/contrib/miniselect/README.md b/contrib/miniselect/README.md deleted file mode 100644 index cbe576ddba8..00000000000 --- a/contrib/miniselect/README.md +++ /dev/null @@ -1,272 +0,0 @@ -[![Build Status](https://travis-ci.com/danlark1/miniselect.svg?branch=main)](https://travis-ci.com/danlark1/miniselect) -[![License](https://img.shields.io/badge/License-Boost%201.0-lightblue.svg)](https://www.boost.org/LICENSE_1_0.txt) - -miniselect : Generic selection and partial ordering algorithms -============================================================== - -`miniselect` is a C++ header-only library that contains various generic selection -and partial sorting algorithms with the ease of use, testing, advice on usage and -benchmarking. - -Sorting is everywhere and there are many outstanding sorting algorithms that -compete in speed, comparison count and cache friendliness. However selection -algorithms are always a bit outside of the competition scope, however they are -pretty important, for example, in databases ORDER BY LIMIT N is used extremely -often which can benefit from more optimal selection and partial sorting -algorithms. This library tries to solve this problem with Modern C++. - -* **Easy:** First-class, easy to use dependency and carefully documented APIs and algorithm properties. -* **Fast:** We do care about speed of the algorithms and provide reasonable implementations. -* **Standard compliant:** We provide C++11 compatible APIs that are compliant to the standard [`std::nth_element`](https://en.cppreference.com/w/cpp/algorithm/nth_element) and [`std::partial_sort`](https://en.cppreference.com/w/cpp/algorithm/partial_sort) functions including custom comparators and order guarantees. Just replace the names of the functions in your project and it should work! -* **Well tested:** We test all algorithms with a unified framework, under sanitizers and fuzzing. -* **Benchmarked:** We gather benchmarks for all implementations to better understand good and bad spots. - -Table of Contents ------------------ - -* [Quick Start](#quick-start) -* [Testing](#testing) -* [Documentation](#documentation) -* [Performance results](#performance-results) -* [Real-world usage](#real-world-usage) -* [Contributing](#contributing) -* [Motivation](#motivation) -* [License](#license) - -Quick Start ------------ - -You can either include this project as a cmake dependency and then use the -headers that are provided in the [include](./include) folder or just pass the -[include](./include) folder to your compiler. - -```cpp -#include -#include - -#include "miniselect/median_of_ninthers.h" - -int main() { - std::vector v = {1, 8, 4, 3, 2, 9, 0, 7, 6, 5}; - miniselect::median_of_ninthers_select(v.begin(), v.begin() + 5, v.end()); - for (const int i : v) { - std::cout << i << ' '; - } - return 0; -} -// Compile it `clang++/g++ -I$DIRECTORY/miniselect/include/ example.cpp -std=c++11 -O3 -o example -// Possible output: 0 1 4 3 2 5 8 7 6 9 -``` - -Examples can be found in [examples](./examples). - -We support all compilers starting from GCC 7 and Clang 6. We are also planning -to support Windows, for now it is best effort but no issues are known so far. - -More on which algorithms are available, see [documentation](#documentation). - -Testing -------- - -To test and benchmark, we use [Google benchmark](https://github.com/google/benchmark) library. -Simply do in the root directory: - -```console -# Check out the library. -$ git clone https://github.com/google/benchmark.git -# Benchmark requires Google Test as a dependency. Add the source tree as a subdirectory. -$ git clone https://github.com/google/googletest.git benchmark/googletest -$ mkdir build && cd build -$ cmake -DMINISELECT_TESTING=on .. -$ make -j -$ ctest -j4 --output-on-failure -``` - -It will create two tests and two benchmarks `test_sort`, `test_select`, -`benchmark_sort`, `benchmark_select`. Use them to validate or contribute. You -can also use `ctest` - -Documentation -------------- - -There are several selection algorithms available, further ![\large n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n) is the number -of elements in the array, ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) is the selection element that is needed to be found (all algorithms are deterministic and not stable unless otherwise is specified): - - -| Name | Average | Best Case | Worst Case | Comparisons | Memory | -|------------------------- |--------------------------------------------------------------------------------------------------------- |--------------------------------------------------------------------------------------------------------- |----------------------------------------------------------------------------------------------------------------------- |---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |--------------------------------------------------------------------------------------------------------------------------------- | -| [pdqselect](./include/miniselect/pdqselect.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+n%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 2.5n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2.5n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | -| [Floyd-Rivest](./include/miniselect/floyd_rivest_select.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | Avg: ![\large n + \min(k, n - k) + O(\sqrt{n \log n})](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n+%2B+%5Cmin%28k%2C+n+-+k%29+%2B+O%28%5Csqrt%7Bn+%5Clog+n%7D%29) | ![\large O(\log \log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+%5Clog+n%29) | -| [Median Of Medians](./include/miniselect/median_of_medians.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | Between ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) and ![\large 22n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+22n). Random data ![\large 2.5n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2.5n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | -| [Median Of Ninthers](./include/miniselect/median_of_ninthers.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | Between ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) and ![\large 12n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+12.5n). Random data ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | -| [Median Of 3 Random](./include/miniselect/median_of_3_random.h) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) | -| [libstdc++ (introselect)](https://github.com/gcc-mirror/gcc/blob/e0af865ab9d9d5b6b3ac7fdde26cf9bbf635b6b4/libstdc%2B%2B-v3/include/bits/stl_algo.h#L4748) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+n%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | -| [libc++ (median of 3)](https://github.com/llvm/llvm-project/blob/3ed89b51da38f081fedb57727076262abb81d149/libcxx/include/algorithm#L5159) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n^2 )](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5E2+%29) | At least ![\large 2n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+2n). Random data ![\large 3n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+3n) | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | - -For sorting the situation is similar except every line adds ![\large O(k\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28k%5Clog+k%29) comparisons and pdqselect is using ![\large O(\log n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28%5Clog+n%29) memory with one more general exception called partial sorting in C++ standard library. - -| Name | Average | Best Case | Worst Case | Comparisons | Memory | -|-------------------|-----------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------- | -| [std::partial_sort](https://github.com/llvm/llvm-project/blob/3ed89b51da38f081fedb57727076262abb81d149/libcxx/include/algorithm#L5074) | ![\large O(n\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+k%29) | ![\large O(n)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%29) | ![\large O(n\log k)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%28n%5Clog+k%29) | ![\large n\log k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n%5Clog+k) on average, for some data patterns might be better | ![\large O(1)](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+O%281%29) | - -## API - -All functions end either in `select`, either in `partial_sort` and -their behavior is exactly the same as for -[`std::nth_element`](https://en.cppreference.com/w/cpp/algorithm/nth_element) -and [`std::partial_sort`](https://en.cppreference.com/w/cpp/algorithm/partial_sort) -respectively, i.e. they accept 3 arguments as `first`, `middle`, `end` iterators -and an optional comparator. Several notes: - -* You should not throw exceptions from `Compare` function. Standard library -also does not specify the behavior in that matter. -* We don't support ParallelSTL for now. -* C++20 constexpr specifiers might be added but currently we don't have them -because of some floating point math in several algorithms. -* All functions are in the `miniselect` namespace. See the example for that. - -- pdqselect - - This algorithm is based on [`pdqsort`](https://github.com/orlp/pdqsort) which is acknowledged as one of the fastest generic sort algorithms. - - **Location:** [`miniselect/pdqselect.h`](./include/miniselect/pdqselect.h). - - **Functions:** `pdqselect`, `pdqselect_branchless`, `pdqpartial_sort`, `pdqpartial_sort_branchless`. Branchless version uses branchless partition algorithm provided by [`pdqsort`](https://github.com/orlp/pdqsort). Use it if your comparison function is branchless, it might give performance for very big ranges. - - **Performance advice:** Use it when you need to sort a big chunk so that ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) is close to ![\large n](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+n). - -

- -- Floyd-Rivest - - This algorithm is based on [Floyd-Rivest algorithm](https://en.wikipedia.org/wiki/Floyd%E2%80%93Rivest_algorithm). - - **Location:** [`miniselect/floyd_rivest_select.h`](./include/miniselect/floyd_rivest_select.h). - - **Functions:** `floyd_rivest_select`, `floyd_rivest_partial_sort`. - - **Performance advice:** Given that this algorithm performs as one of the best on average case in terms of comparisons and speed, we highly advise to - at least try this in your project. Especially it is good for small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) or types that are expensive to compare (for example, strings). But even for median the benchmarks show it outperforms others. It is not easy for this algorithm to build a reasonable worst case but one of examples when this algorithm does not perform well is when there are lots of similar values of linear size (random01 dataset showed some moderate penalties). - -We present here two gifs, for median and for ![\large k = n / 10](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k+%3D+n+%2F+10) order statistic. - -

- - -

- -- Median Of Medians - - This algorithm is based on [Median of Medians](https://en.wikipedia.org/wiki/Median_of_medians) algorithm, one of the first deterministic linear time worst case median algorithm - - **Location:** [`miniselect/median_of_medians.h`](./include/miniselect/median_of_medians.h). - - **Functions:** `median_of_medians_select`, `median_of_medians_partial_sort`. - - **Performance advice:** This algorithm does not show advantages over others, implemented for historical reasons and for bechmarking. - -

- -- Median Of Ninthers - - This algorithm is based on [Fast Deterministic Selection](https://erdani.com/research/sea2017.pdf) paper by Andrei Alexandrescu, one of the latest and fastest deterministic linear time worst case median algorithms - - **Location:** [`miniselect/median_of_ninthers.h`](./include/miniselect/median_of_ninthers.h). - - **Functions:** `median_of_ninthers_select`, `median_of_ninthers_partial_sort`. - - **Performance advice:** Use this algorithm if you absolutely need linear time worst case scenario for selection algorithm. This algorithm shows some strengths over other deterministic [`PICK`](https://en.wikipedia.org/wiki/Median_of_medians) algorithms and has lower constanst than MedianOfMedians. - -

- -- Median Of 3 Random - - This algorithm is based on QuickSelect with the random median of 3 pivot choice algorithm (it chooses random 3 elements in the range and takes the middle value). It is a rando - - **Location:** [`miniselect/median_of_3_random.h`](./include/miniselect/median_of_3_random.h). - - **Functions:** `median_of_3_random_select`, `median_of_3_random_partial_sort`. - - **Performance advice:** This is a randomized algorithm and also it did not show any strengths against Median Of Ninthers. - -

- -- Introselect - - This algorithm is based on [Introselect](https://en.wikipedia.org/wiki/Introselect) algorithm, it is used in libstdc++ in `std::nth_element`, however instead of falling back to MedianOfMedians it is using HeapSelect which adds logarithm to its worst complexity. - - **Location:** ``. - - **Functions:** `std::nth_element`. - - **Performance advice:** This algorithm is used in standard library and is not recommended to use if you are looking for performance. - -

- -- Median Of 3 - - This algorithm is based on QuickSelect with median of 3 pivot choice algorithm (the middle value between begin, mid and end values), it is used in libc++ in `std::nth_element`. - - **Location:** ``. - - **Functions:** `std::nth_element`. - - **Performance advice:** This algorithm is used in standard library and is not recommended to use if you are looking for performance. - -

- -- `std::partial_sort` - - This algorithm has [heap-based solutions](https://en.wikipedia.org/wiki/Partial_sorting) both in libc++ and libstdc++, from the first ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) elements the max heap is built, then one by one the elements are trying to be pushed to that heap with HeapSort in the end. - - **Location:** ``. - - **Functions:** `std::partial_sort`. - - **Performance advice:** This algorithm is very good for random data and small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) and might outperform all selection+sort algorithms. However, for descending data it starts to significantly degrade and is not recommended for use if you have such patterns in real data. - -

- -## Other algorithms to come - -* Kiwiel modification of FloydRivest algorithm which is described in [On Floyd and Rivest’s SELECT algorithm](https://core.ac.uk/download/pdf/82672439.pdf) with ternary and quintary pivots. -* Combination of FloydRivest and pdqsort pivot strategies, currently all experiments did not show any boost. - -Performance results -------------------- - -We use 10 datasets and 8 algorithms with 10000000 elements to find median and -other ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) on `Intel(R) Core(TM) i5-4200H CPU @ 2.80GHz` for `std::vector`, -for median the benchmarks are the following: - -![median](benches/plots/result_10000000_5000000.png) - -![median](benches/plots/result_comparisons_10000000_5000000.png) - -For smaller ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k), -for example, 1000, the results are the following - -![k equals 1000](benches/plots/result_10000000_1000.png) - -![k equals 1000](benches/plots/result_comparisons_10000000_1000.png) - -Other benchmarks can be found [here](https://drive.google.com/drive/folders/1DHEaeXgZuX6AJ9eByeZ8iQVQv0ueP8XM). - -The benchmarks for number of swaps will be later. - -Real-world usage ----------------- - -- [Yandex ClickHouse](https://github.com/yandex/ClickHouse) - -If you are planning to use miniselect in your product, please work from one of -our releases and if you wish, you can write the acknowledgment in this section -for visibility. - -Contributing ------------- - -Patches are welcome with new algorithms! You should add the selection algorithm -together with the partial sorting algorithm in [include](./include), add -tests in [testing](./testing) and ideally run benchmarks to see how it performs. -If you also have some data cases to test against, we would be more than happy -to merge them. - -Motivation ----------- - -Firstly the author was interested if any research had been done for small ![\large k](https://render.githubusercontent.com/render/math?math=%5Cdisplaystyle+%5Clarge+k) -in selection algorithms and was struggling to find working implementations to -compare different approaches from standard library and quickselect algorithms. -After that it turned out that the problem is much more interesting than it looks -like and after reading The Art of Computer Programming from Donald Knuth about -minimum comparison sorting and selection algorithms the author decided to look -through all non-popular algorithms and try them out. - -The author have not found any decent library for selection algorithms and little -research is published in open source, so that they decided to merge all that -implementations and compare them with possible merging of different ideas -into a decent one algorithm for most needs. For a big story of adventures see -the author's blog post TODO. - -License -------- - -The code is made available under the [Boost License 1.0](https://boost.org/LICENSE_1_0.txt). - -Third-Party Libraries Used and Adjusted ---------------------------------------- - -| Library | License | -|---------------------|--------------------------------------------------------------------------------------------------| -| pdqsort | [MIT](https://github.com/orlp/pdqsort/blob/47a46767d76fc852284eaa083e4b7034ee6e2559/license.txt) | -| MedianOfNinthers | [Boost License 1.0](https://github.com/andralex/MedianOfNinthers/blob/master/LICENSE_1_0.txt) | - diff --git a/contrib/miniselect/benches/bench_common.h b/contrib/miniselect/benches/bench_common.h deleted file mode 100644 index b49b55dac9d..00000000000 --- a/contrib/miniselect/benches/bench_common.h +++ /dev/null @@ -1,170 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include -#include -#include - -namespace miniselect { -namespace datagens { - -struct Random { - static std::vector Gen(size_t size) { - std::random_device rnd_device; - std::mt19937_64 mersenne_engine{rnd_device()}; - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size; ++i) { - v.push_back(i); - } - std::shuffle(v.begin(), v.end(), mersenne_engine); - return v; - } -}; - -struct Shuffled16 { - static std::vector Gen(size_t size) { - std::random_device rnd_device; - std::mt19937_64 mersenne_engine{rnd_device()}; - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size; ++i) { - v.push_back(i % 16); - } - std::shuffle(v.begin(), v.end(), mersenne_engine); - return v; - } -}; - -struct Random01 { - static std::vector Gen(size_t size) { - std::random_device rnd_device; - std::mt19937_64 mersenne_engine{rnd_device()}; - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size; ++i) { - v.push_back(i % 2); - } - std::shuffle(v.begin(), v.end(), mersenne_engine); - return v; - } -}; - -struct Ascending { - static std::vector Gen(size_t size) { - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size; ++i) { - v.push_back(i); - } - return v; - } -}; - -struct Descending { - static std::vector Gen(size_t size) { - std::vector v; - v.reserve(size); - for (int i = size - 1; i >= 0; --i) { - v.push_back(i); - } - return v; - } -}; - -struct PipeOrgan { - static std::vector Gen(size_t size) { - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size / 2; ++i) { - v.push_back(i); - } - for (size_t i = size / 2; i < size; ++i) { - v.push_back(size - i); - } - return v; - } -}; - -struct PushFront { - static std::vector Gen(size_t size) { - std::vector v; - v.reserve(size); - for (size_t i = 1; i < size; ++i) { - v.push_back(i); - } - v.push_back(0); - return v; - } -}; - -struct PushMiddle { - static std::vector Gen(size_t size) { - std::vector v; - v.reserve(size); - for (size_t i = 0; i < size; ++i) { - if (i != size / 2) { - v.push_back(i); - } - } - v.push_back(size / 2); - return v; - } -}; - -struct Median3Killer { - static std::vector Gen(size_t size) { - size_t k = size / 2; - std::vector v; - v.reserve(size); - for (size_t i = 1; i < k + 1; ++i) { - if (i & 1) { - v.push_back(i); - } else { - v.push_back(k + i - 1); - } - } - for (size_t i = 1; i < k + 1; ++i) { - v.push_back(2 * i); - } - return v; - } -}; - -#define BENCH_IMPL(BENCH, GEN, IMPL) \ - BENCHMARK_TEMPLATE(BENCH, GEN, IMPL) \ - ->Unit(benchmark::kMicrosecond) \ - ->Arg(kSize - 10) \ - ->Arg(kSize / 2) \ - ->Arg(10000) \ - ->Arg(1000) \ - ->Arg(100) \ - ->Arg(10) \ - ->Arg(1) - -#define BENCH_GENS(BENCH, IMPL) \ - BENCH_IMPL(BENCH, datagens::Random, IMPL); \ - BENCH_IMPL(BENCH, datagens::Shuffled16, IMPL); \ - BENCH_IMPL(BENCH, datagens::Random01, IMPL); \ - BENCH_IMPL(BENCH, datagens::Ascending, IMPL); \ - BENCH_IMPL(BENCH, datagens::Descending, IMPL); \ - BENCH_IMPL(BENCH, datagens::PipeOrgan, IMPL); \ - BENCH_IMPL(BENCH, datagens::PushMiddle, IMPL); \ - BENCH_IMPL(BENCH, datagens::PushFront, IMPL); \ - BENCH_IMPL(BENCH, datagens::Median3Killer, IMPL) - -#define BENCH(NAME) \ - BENCH_GENS(NAME, algorithms::FloydRivest); \ - BENCH_GENS(NAME, algorithms::MedianOfNinthers); \ - BENCH_GENS(NAME, algorithms::MedianOfMedians); \ - BENCH_GENS(NAME, algorithms::MedianOf3Random); \ - BENCH_GENS(NAME, algorithms::PDQ); \ - BENCH_GENS(NAME, algorithms::PDQBranchless); \ - BENCH_GENS(NAME, algorithms::STD) - -} // namespace datagens -} // namespace miniselect diff --git a/contrib/miniselect/benches/benchmark_select.cpp b/contrib/miniselect/benches/benchmark_select.cpp deleted file mode 100644 index 2a9b238c90a..00000000000 --- a/contrib/miniselect/benches/benchmark_select.cpp +++ /dev/null @@ -1,46 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#include - -#include -#include -#include -#include -#include -#include - -#include "bench_common.h" -#include "test_common.h" - -namespace miniselect { -namespace { - -static constexpr size_t kSize = 65536; - -template -static void BM_sel(benchmark::State& state) { - auto vec = DataGen::Gen(kSize); - const size_t arg = state.range(0); - size_t cnt = 0; - size_t cmp = 0; - for (auto _ : state) { - Impl::Select(vec.begin(), vec.begin() + arg, vec.end(), - [&cmp](const auto& left, const auto& right) { - cmp++; - return left < right; - }); - ++cnt; - benchmark::DoNotOptimize(vec[arg]); - } - state.counters["Comparisons"] = 1.0 * cmp / cnt; -} - -BENCH(BM_sel); - -} // namespace -} // namespace miniselect - -BENCHMARK_MAIN(); diff --git a/contrib/miniselect/benches/benchmark_sort.cpp b/contrib/miniselect/benches/benchmark_sort.cpp deleted file mode 100644 index 8b3bbd1a77f..00000000000 --- a/contrib/miniselect/benches/benchmark_sort.cpp +++ /dev/null @@ -1,46 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#include - -#include -#include -#include -#include -#include -#include - -#include "bench_common.h" -#include "test_common.h" - -namespace miniselect { -namespace { - -static constexpr size_t kSize = 65536; - -template -static void BM_sort(benchmark::State& state) { - auto vec = DataGen::Gen(kSize); - const size_t arg = state.range(0); - size_t cnt = 0; - size_t cmp = 0; - for (auto _ : state) { - Impl::Sort(vec.begin(), vec.begin() + arg, vec.end(), - [&cmp](const auto& left, const auto& right) { - cmp++; - return left < right; - }); - ++cnt; - benchmark::DoNotOptimize(vec[arg]); - } - state.counters["Comparisons"] = 1.0 * cmp / cnt; -} - -BENCH(BM_sort); - -} // namespace -} // namespace miniselect - -BENCHMARK_MAIN(); diff --git a/contrib/miniselect/examples/example.cpp b/contrib/miniselect/examples/example.cpp deleted file mode 100644 index 183e81ae1b6..00000000000 --- a/contrib/miniselect/examples/example.cpp +++ /dev/null @@ -1,18 +0,0 @@ -#include -#include - -#include "miniselect/median_of_ninthers.h" - -int main() { - std::vector v = {1, 8, 4, 3, 2, 9, 0, 7, 6, 5}; - miniselect::median_of_ninthers_select(v.begin(), v.begin() + 5, v.end()); - for (const int i : v) { - std::cout << i << ' '; - } - return 0; -} - -// Compile it `clang++/g++ -I$DIRECTORY/miniselect/include/ example.cpp -std=c++11 -O3 -o example - -// Possible output: 0 1 4 3 2 5 8 7 6 9 -// ^ on the right place diff --git a/contrib/miniselect/fuzz/CMakeLists.txt b/contrib/miniselect/fuzz/CMakeLists.txt deleted file mode 100644 index 38473bd78ad..00000000000 --- a/contrib/miniselect/fuzz/CMakeLists.txt +++ /dev/null @@ -1,38 +0,0 @@ -cmake_minimum_required(VERSION 3.7) - -project(fuzz) - -option(ENABLE_FUZZING "enable building the fuzzers" ON) -set(CMAKE_CXX_STANDARD 17) - -if(ENABLE_FUZZING) - set(MINISELECT_FUZZ_LDFLAGS "" CACHE STRING "LDFLAGS for the fuzz targets") - - add_library(miniselect-fuzzer INTERFACE) - target_link_libraries(miniselect-fuzzer INTERFACE gtest) - target_link_libraries(miniselect-fuzzer INTERFACE ${MINISELECT_FUZZ_LDFLAGS}) - - if(MINISELECT_FUZZ_LINKMAIN) - target_sources(simdjson-fuzzer INTERFACE $/main.cpp) - endif() - - # Define the fuzzers - add_custom_target(all_fuzzers) - - set(fuzzernames) - function(implement_fuzzer name) - add_executable(${name} ${name}.cpp) - target_link_libraries(${name} PRIVATE miniselect-fuzzer) - add_dependencies(all_fuzzers ${name}) - set(fuzzernames ${fuzzernames} ${name} PARENT_SCOPE) - endfunction() - - implement_fuzzer(fuzz_select) - implement_fuzzer(fuzz_string_select) - implement_fuzzer(fuzz_sort) - implement_fuzzer(fuzz_string_sort) - - # to be able to get a list of all fuzzers from within a script - add_custom_target(print_all_fuzzernames - COMMAND ${CMAKE_COMMAND} -E echo ${fuzzernames}) -endif() diff --git a/contrib/miniselect/fuzz/build_like_oss_fuzz.sh b/contrib/miniselect/fuzz/build_like_oss_fuzz.sh deleted file mode 100755 index 547348133a9..00000000000 --- a/contrib/miniselect/fuzz/build_like_oss_fuzz.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/bin/sh -# -# This script emulates how oss fuzz invokes the build -# process, handy for trouble shooting cmake issues and possibly -# recreating testcases. For proper debugging of the oss fuzz -# build, follow the procedure at https://google.github.io/oss-fuzz/getting-started/new-project-guide/#testing-locally - -set -eu - -ossfuzz=$(readlink -f $(dirname $0))/ossfuzz.sh - -mkdir -p ossfuzz-out -export OUT=$(pwd)/ossfuzz-out -export CC=clang -export CXX="clang++" -export CFLAGS="-fsanitize=fuzzer-no-link" -export CXXFLAGS="-fsanitize=fuzzer-no-link,address,undefined -O1" -export LIB_FUZZING_ENGINE="-fsanitize=fuzzer" - -$ossfuzz - -echo "look at the results in $OUT" diff --git a/contrib/miniselect/fuzz/fuzz_select.cpp b/contrib/miniselect/fuzz/fuzz_select.cpp deleted file mode 100644 index f70980bd0d9..00000000000 --- a/contrib/miniselect/fuzz/fuzz_select.cpp +++ /dev/null @@ -1,66 +0,0 @@ -#include -#include -#include -#include - -#include "test_common.h" - -template -void ChooseImplementation(uint8_t byte, std::vector& working, - Iter partition_iter, const ::testing::Types&) { - static_assert(sizeof...(T) < 256); - int i = 0; - constexpr size_t size = sizeof...(T); - ( - [&]() { - if (byte % size == i++) { - T::Select(working.begin(), partition_iter, working.end()); - } - }(), - ...); -} - -// Use the first element as a position into the data -extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, - std::size_t size) { - if (size <= 3) return 0; - uint8_t impl = data[0]; - uint16_t partition_point = 0; - memcpy(&partition_point, data + 1, 2); - partition_point %= (size - 3); - std::vector working(data + 3, data + size); - auto canonical = working; - const auto partition_iter = working.begin() + partition_point; - ChooseImplementation(impl, working, partition_iter, - miniselect::algorithms::All{}); - - if (partition_iter != working.end()) { - const auto& nth = *partition_iter; - bool is_error = false; - if (!std::all_of(working.begin(), partition_iter, - [&](const auto& v) { return v <= nth; })) { - is_error = true; - } - if (!std::all_of(partition_iter, working.end(), - [&](const auto& v) { return v >= nth; })) { - is_error = true; - } - if (is_error) { - std::cerr << "FAILED!\nCanonical: "; - for (const auto& s : canonical) { - std::cerr << static_cast(s) << ' '; - } - std::cerr << std::endl; - std::cerr << "Got: "; - for (const auto& s : working) { - std::cerr << static_cast(s) << ' '; - } - std::cerr << std::endl; - std::cerr << "partition_iter = " << partition_iter - working.begin() - << std::endl; - std::abort(); - } - } - - return 0; -} diff --git a/contrib/miniselect/fuzz/fuzz_sort.cpp b/contrib/miniselect/fuzz/fuzz_sort.cpp deleted file mode 100644 index ba0a2b6ca3e..00000000000 --- a/contrib/miniselect/fuzz/fuzz_sort.cpp +++ /dev/null @@ -1,69 +0,0 @@ -#include -#include -#include -#include - -#include "test_common.h" - -template -void ChooseImplementation(uint8_t byte, std::vector& working, - Iter partition_iter, const ::testing::Types&) { - static_assert(sizeof...(T) < 256); - int i = 0; - constexpr size_t size = sizeof...(T); - ( - [&]() { - if (byte % size == i++) { - T::Sort(working.begin(), partition_iter, working.end()); - } - }(), - ...); -} - -// Use the first element as a position into the data -extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, - std::size_t size) { - if (size <= 3) return 0; - uint8_t impl = data[0]; - uint16_t partition_point = 0; - memcpy(&partition_point, data + 1, 2); - partition_point %= (size - 3); - std::vector working(data + 3, data + size); - auto canonical = working; - const auto partition_iter = working.begin() + partition_point; - ChooseImplementation(impl, working, partition_iter, - miniselect::algorithms::All{}); - - bool is_error = false; - if (partition_iter != working.end()) { - const auto& nth = *std::min_element(partition_iter, working.end()); - if (!std::all_of(working.begin(), partition_iter, - [&](const auto& v) { return v <= nth; })) { - is_error = true; - } - if (!std::all_of(partition_iter, working.end(), - [&](const auto& v) { return v >= nth; })) { - is_error = true; - } - } - if (!std::is_sorted(working.begin(), partition_iter)) { - is_error = true; - } - if (is_error) { - std::cerr << "FAILED!\nCanonical: "; - for (const auto& s : canonical) { - std::cerr << static_cast(s) << ' '; - } - std::cerr << std::endl; - std::cerr << "Got: "; - for (const auto& s : working) { - std::cerr << static_cast(s) << ' '; - } - std::cerr << std::endl; - std::cerr << "partition_iter = " << partition_iter - working.begin() - << std::endl; - std::abort(); - } - - return 0; -} diff --git a/contrib/miniselect/fuzz/fuzz_string_select.cpp b/contrib/miniselect/fuzz/fuzz_string_select.cpp deleted file mode 100644 index cd24b376d86..00000000000 --- a/contrib/miniselect/fuzz/fuzz_string_select.cpp +++ /dev/null @@ -1,70 +0,0 @@ -#include -#include -#include -#include - -#include "test_common.h" - -template -void ChooseImplementation(uint8_t byte, std::vector& working, - Iter partition_iter, const ::testing::Types&) { - static_assert(sizeof...(T) < 256); - int i = 0; - constexpr size_t size = sizeof...(T); - ( - [&]() { - if (byte % size == i++) { - T::Select(working.begin(), partition_iter, working.end()); - } - }(), - ...); -} - -// Use the first element as a position into the data -extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, - std::size_t size) { - if (size <= 3) return 0; - uint8_t impl = data[0]; - uint16_t partition_point = 0; - memcpy(&partition_point, data + 1, 2); - partition_point %= (size - 3); - std::vector working; - for (auto i = data + 3; i < data + size; ++i) { - std::string s(1, *i); - working.push_back(s); - } - auto canonical = working; - const auto partition_iter = working.begin() + partition_point; - ChooseImplementation(impl, working, partition_iter, - miniselect::algorithms::All{}); - // nth may be the end iterator, in this case nth_element has no effect. - if (partition_iter != working.end()) { - const auto& nth = *partition_iter; - bool is_error = false; - if (!std::all_of(working.begin(), partition_iter, - [&](const auto& v) { return v <= nth; })) { - is_error = true; - } - if (!std::all_of(partition_iter, working.end(), - [&](const auto& v) { return v >= nth; })) { - is_error = true; - } - if (is_error) { - std::cerr << "FAILED!\nCanonical: "; - for (const auto& s : canonical) { - std::cerr << s << ' '; - } - std::cerr << std::endl; - std::cerr << "Got: "; - for (const auto& s : working) { - std::cerr << s << ' '; - } - std::cerr << std::endl; - std::cerr << "partition_iter = " << partition_iter - working.begin() - << std::endl; - std::abort(); - } - } - - return 0; -} diff --git a/contrib/miniselect/fuzz/fuzz_string_sort.cpp b/contrib/miniselect/fuzz/fuzz_string_sort.cpp deleted file mode 100644 index a797e0d7e22..00000000000 --- a/contrib/miniselect/fuzz/fuzz_string_sort.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include -#include -#include - -#include "test_common.h" - -template -void ChooseImplementation(uint8_t byte, std::vector& working, - Iter partition_iter, const ::testing::Types&) { - static_assert(sizeof...(T) < 256); - int i = 0; - constexpr size_t size = sizeof...(T); - ( - [&]() { - if (byte % size == i++) { - T::Sort(working.begin(), partition_iter, working.end()); - } - }(), - ...); -} - -// Use the first element as a position into the data -extern "C" int LLVMFuzzerTestOneInput(const std::uint8_t* data, - std::size_t size) { - if (size <= 3) return 0; - uint8_t impl = data[0]; - uint16_t partition_point = 0; - memcpy(&partition_point, data + 1, 2); - partition_point %= (size - 3); - std::vector working; - for (auto i = data + 3; i < data + size; ++i) { - std::string s(1, *i); - working.push_back(s); - } - auto canonical = working; - const auto partition_iter = working.begin() + partition_point; - ChooseImplementation(impl, working, partition_iter, - miniselect::algorithms::All{}); - // nth may be the end iterator, in this case nth_element has no effect. - bool is_error = false; - if (partition_iter != working.end()) { - const auto& nth = *std::min_element(partition_iter, working.end()); - if (!std::all_of(working.begin(), partition_iter, - [&](const auto& v) { return v <= nth; })) { - is_error = true; - } - if (!std::all_of(partition_iter, working.end(), - [&](const auto& v) { return v >= nth; })) { - is_error = true; - } - } - if (!std::is_sorted(working.begin(), partition_iter)) { - is_error = true; - } - if (is_error) { - std::cerr << "FAILED!\nCanonical: "; - for (const auto& s : canonical) { - std::cerr << s << ' '; - } - std::cerr << std::endl; - std::cerr << "Got: "; - for (const auto& s : working) { - std::cerr << s << ' '; - } - std::cerr << std::endl; - std::cerr << "partition_iter = " << partition_iter - working.begin() - << std::endl; - std::abort(); - } - - return 0; -} diff --git a/contrib/miniselect/fuzz/main.cpp b/contrib/miniselect/fuzz/main.cpp deleted file mode 100644 index e3377035f33..00000000000 --- a/contrib/miniselect/fuzz/main.cpp +++ /dev/null @@ -1,22 +0,0 @@ -#include -#include -#include -#include - -extern "C" int LLVMFuzzerTestOneInput(const uint8_t* Data, std::size_t Size); - -int main(int argc, char* argv[]) { - for (int i = 1; i < argc; ++i) { - std::ifstream in(argv[i]); - assert(in); - in.seekg(0, std::ios_base::end); - const auto pos = in.tellg(); - assert(pos >= 0); - in.seekg(0, std::ios_base::beg); - std::vector buf(static_cast(pos)); - in.read(buf.data(), static_cast(buf.size())); - assert(in.gcount() == pos); - LLVMFuzzerTestOneInput(reinterpret_cast(buf.data()), - buf.size()); - } -} diff --git a/contrib/miniselect/fuzz/ossfuzz.sh b/contrib/miniselect/fuzz/ossfuzz.sh deleted file mode 100755 index 83f37f54eee..00000000000 --- a/contrib/miniselect/fuzz/ossfuzz.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/sh -# -# entry point for oss-fuzz, so that fuzzers -# and build invocation can be changed without having -# to modify the oss-fuzz repo. -# -# invoke it from the git root. - -# make sure to exit on problems -set -eux - -mkdir -p build -cd build - -cmake .. \ --GNinja \ --DCMAKE_BUILD_TYPE=Debug \ --DENABLE_FUZZING=On \ --DMINISELECT_FUZZ_LINKMAIN=off \ --DMINISELECT_FUZZ_LDFLAGS=$LIB_FUZZING_ENGINE - -cmake --build . --target all_fuzzers - diff --git a/contrib/miniselect/include/miniselect/floyd_rivest_select.h b/contrib/miniselect/include/miniselect/floyd_rivest_select.h deleted file mode 100644 index e7d5f80f572..00000000000 --- a/contrib/miniselect/include/miniselect/floyd_rivest_select.h +++ /dev/null @@ -1,120 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace miniselect { -namespace floyd_rivest_detail { - -template -struct CompareRefType { - // Pass the comparator by lvalue reference. Or in debug mode, using a - // debugging wrapper that stores a reference. - using type = typename std::add_lvalue_reference::type; -}; - -template -inline void floyd_rivest_select_loop(Iter begin, Diff left, Diff right, Diff k, - Compare comp) { - while (right > left) { - Diff size = right - left; - if (size > 600) { - Diff n = right - left + 1; - Diff i = k - left + 1; - double z = log(n); - double s = 0.5 * exp(2 * z / 3); - double sd = 0.5 * sqrt(z * s * (n - s) / n); - if (i < n / 2) { - sd *= -1.0; - } - Diff newLeft = std::max(left, (Diff)(k - i * s / n + sd)); - Diff newRight = std::min(right, (Diff)(k + (n - i) * s / n + sd)); - floyd_rivest_select_loop(begin, newLeft, newRight, k, - comp); - } - Diff i = left; - Diff j = right; - std::swap(begin[left], begin[k]); - const bool to_swap = comp(begin[left], begin[right]); - if (to_swap) { - std::swap(begin[left], begin[right]); - } - // Make sure that non copyable types compile. - const auto& t = to_swap ? begin[left] : begin[right]; - while (i < j) { - std::swap(begin[i], begin[j]); - i++; - j--; - while (comp(begin[i], t)) { - i++; - } - while (comp(t, begin[j])) { - j--; - } - } - - if (to_swap) { - std::swap(begin[left], begin[j]); - } else { - j++; - std::swap(begin[right], begin[j]); - } - - if (j <= k) { - left = j + 1; - } - if (k <= j) { - right = j - 1; - } - } -} - -} // namespace floyd_rivest_detail - -template -inline void floyd_rivest_partial_sort(Iter begin, Iter mid, Iter end, - Compare comp) { - if (begin == end) return; - if (begin == mid) return; - using CompType = typename floyd_rivest_detail::CompareRefType::type; - - floyd_rivest_detail::floyd_rivest_select_loop< - Iter, CompType, typename std::iterator_traits::difference_type>( - begin, 0, end - begin - 1, mid - begin - 1, comp); - // std::sort proved to be better than other sorts because of pivoting. - std::sort(begin, mid, comp); -} - -template -inline void floyd_rivest_partial_sort(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - floyd_rivest_partial_sort(begin, mid, end, std::less()); -} - -template -inline void floyd_rivest_select(Iter begin, Iter mid, Iter end, Compare comp) { - if (mid == end) return; - using CompType = typename floyd_rivest_detail::CompareRefType::type; - - floyd_rivest_detail::floyd_rivest_select_loop< - Iter, CompType, typename std::iterator_traits::difference_type>( - begin, 0, end - begin - 1, mid - begin, comp); -} - -template -inline void floyd_rivest_select(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - floyd_rivest_select(begin, mid, end, std::less()); -} - -} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_3_random.h b/contrib/miniselect/include/miniselect/median_of_3_random.h deleted file mode 100644 index 0f7b62fd61c..00000000000 --- a/contrib/miniselect/include/miniselect/median_of_3_random.h +++ /dev/null @@ -1,69 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include -#include -#include -#include -#include - -#include "private/median_common.h" - -namespace miniselect { -namespace median_of_3_random_detail { - -template -static inline Iter partition(Iter r, Iter end, Compare&& comp) { - typedef typename std::iterator_traits::difference_type T; - const T len = end - r; - assert(len >= 3); - static std::mt19937_64 gen(1); - std::uniform_int_distribution dis(0, len - 1); - T x = dis(gen); - T y = dis(gen); - T z = dis(gen); - return median_common_detail::pivotPartition( - r, median_common_detail::medianIndex(r, x, y, z, comp), len, comp); -} - -} // namespace median_of_3_random_detail - -template -inline void median_of_3_random_select(Iter begin, Iter mid, Iter end, - Compare comp) { - if (mid == end) return; - using CompType = typename floyd_rivest_detail::CompareRefType::type; - - median_common_detail::quickselect< - Iter, CompType, &median_of_3_random_detail::partition>( - begin, mid, end, comp); -} - -template -inline void median_of_3_random_select(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_3_random_select(begin, mid, end, std::less()); -} - -template -inline void median_of_3_random_sort(Iter begin, Iter mid, Iter end, - Compare comp) { - if (begin == mid) return; - using CompType = typename floyd_rivest_detail::CompareRefType::type; - median_common_detail::quickselect< - Iter, CompType, &median_of_3_random_detail::partition>( - begin, mid - 1, end, comp); - std::sort(begin, mid, comp); -} - -template -inline void median_of_3_random_sort(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_3_random_sort(begin, mid, end, std::less()); -} - -} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_medians.h b/contrib/miniselect/include/miniselect/median_of_medians.h deleted file mode 100644 index 922401b12d0..00000000000 --- a/contrib/miniselect/include/miniselect/median_of_medians.h +++ /dev/null @@ -1,71 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include -#include -#include -#include - -#include "private/median_common.h" - -namespace miniselect { -namespace median_of_medians_detail { - -template -static inline Iter partition(Iter r, Iter end, Compare&& comp) { - using CompType = typename median_common_detail::CompareRefType::type; - const size_t len = end - r; - if (len < 5) { - return median_common_detail::pivotPartition(r, len / 2, len, comp); - } - size_t j = 0; - for (size_t i = 4; i < len; i += 5, ++j) { - median_common_detail::partition5(r, i - 4, i - 3, i, i - 2, i - 1, comp); - std::swap(r[i], r[j]); - } - median_common_detail::quickselect(r, r + j / 2, - r + j, comp); - return median_common_detail::pivotPartition(r, j / 2, len, comp); -} - -} // namespace median_of_medians_detail - -template -inline void median_of_medians_select(Iter begin, Iter mid, Iter end, - Compare comp) { - if (mid == end) return; - using CompType = typename median_common_detail::CompareRefType::type; - - median_common_detail::quickselect< - Iter, CompType, &median_of_medians_detail::partition>( - begin, mid, end, comp); -} - -template -inline void median_of_medians_select(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_medians_select(begin, mid, end, std::less()); -} - -template -inline void median_of_medians_sort(Iter begin, Iter mid, Iter end, - Compare comp) { - if (begin == mid) return; - using CompType = typename median_common_detail::CompareRefType::type; - median_common_detail::quickselect< - Iter, CompType, &median_of_medians_detail::partition>( - begin, mid - 1, end, comp); - std::sort(begin, mid, comp); -} - -template -inline void median_of_medians_sort(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_medians_sort(begin, mid, end, std::less()); -} - -} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/median_of_ninthers.h b/contrib/miniselect/include/miniselect/median_of_ninthers.h deleted file mode 100644 index 099786cf518..00000000000 --- a/contrib/miniselect/include/miniselect/median_of_ninthers.h +++ /dev/null @@ -1,190 +0,0 @@ -/* Copyright Andrei Alexandrescu, 2016-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -// Adjusted from Alexandrescu paper to support arbitrary comparators. -#pragma once - -#include -#include -#include -#include -#include - -#include "private/median_common.h" - -namespace miniselect { -namespace median_of_ninthers_detail { - -template -void adaptiveQuickselect(Iter r, size_t n, size_t length, Compare&& comp); - -/** -Median of minima -*/ -template -size_t medianOfMinima(Iter const r, const size_t n, const size_t length, - Compare&& comp) { - assert(length >= 2); - assert(n * 4 <= length); - assert(n > 0); - const size_t subset = n * 2, computeMinOver = (length - subset) / subset; - assert(computeMinOver > 0); - for (size_t i = 0, j = subset; i < subset; ++i) { - const auto limit = j + computeMinOver; - size_t minIndex = j; - while (++j < limit) - if (comp(r[j], r[minIndex])) minIndex = j; - if (comp(r[minIndex], r[i])) std::swap(r[i], r[minIndex]); - assert(j < length || i + 1 == subset); - } - adaptiveQuickselect(r, n, subset, comp); - return median_common_detail::expandPartition(r, 0, n, subset, length, comp); -} - -/** -Median of maxima -*/ -template -size_t medianOfMaxima(Iter const r, const size_t n, const size_t length, - Compare&& comp) { - assert(length >= 2); - assert(n * 4 >= length * 3 && n < length); - const size_t subset = (length - n) * 2, subsetStart = length - subset, - computeMaxOver = subsetStart / subset; - assert(computeMaxOver > 0); - for (size_t i = subsetStart, j = i - subset * computeMaxOver; i < length; - ++i) { - const auto limit = j + computeMaxOver; - size_t maxIndex = j; - while (++j < limit) - if (comp(r[maxIndex], r[j])) maxIndex = j; - if (comp(r[i], r[maxIndex])) std::swap(r[i], r[maxIndex]); - assert(j != 0 || i + 1 == length); - } - adaptiveQuickselect(r + subsetStart, length - n, subset, comp); - return median_common_detail::expandPartition(r, subsetStart, n, length, - length, comp); -} - -/** -Partitions r[0 .. length] using a pivot of its own choosing. Attempts to pick a -pivot that approximates the median. Returns the position of the pivot. -*/ -template -size_t medianOfNinthers(Iter const r, const size_t length, Compare&& comp) { - assert(length >= 12); - const auto frac = length <= 1024 - ? length / 12 - : length <= 128 * 1024 ? length / 64 : length / 1024; - auto pivot = frac / 2; - const auto lo = length / 2 - pivot, hi = lo + frac; - assert(lo >= frac * 4); - assert(length - hi >= frac * 4); - assert(lo / 2 >= pivot); - const auto gap = (length - 9 * frac) / 4; - auto a = lo - 4 * frac - gap, b = hi + gap; - for (size_t i = lo; i < hi; ++i, a += 3, b += 3) { - median_common_detail::ninther(r, a, i - frac, b, a + 1, i, b + 1, a + 2, - i + frac, b + 2, comp); - } - - adaptiveQuickselect(r + lo, pivot, frac, comp); - return median_common_detail::expandPartition(r, lo, lo + pivot, hi, length, - comp); -} - -/** -Quickselect driver for medianOfNinthers, medianOfMinima, and medianOfMaxima. -Dispathes to each depending on the relationship between n (the sought order -statistics) and length. -*/ -template -void adaptiveQuickselect(Iter r, size_t n, size_t length, Compare&& comp) { - assert(n < length); - for (;;) { - // Decide strategy for partitioning - if (n == 0) { - // That would be the max - auto pivot = n; - for (++n; n < length; ++n) - if (comp(r[n], r[pivot])) pivot = n; - std::swap(r[0], r[pivot]); - return; - } - if (n + 1 == length) { - // That would be the min - auto pivot = 0; - for (n = 1; n < length; ++n) - if (comp(r[pivot], r[n])) pivot = n; - std::swap(r[pivot], r[length - 1]); - return; - } - assert(n < length); - size_t pivot; - if (length <= 16) - pivot = median_common_detail::pivotPartition(r, n, length, comp) - r; - else if (n * 6 <= length) - pivot = medianOfMinima(r, n, length, comp); - else if (n * 6 >= length * 5) - pivot = medianOfMaxima(r, n, length, comp); - else - pivot = medianOfNinthers(r, length, comp); - - // See how the pivot fares - if (pivot == n) { - return; - } - if (pivot > n) { - length = pivot; - } else { - ++pivot; - r += pivot; - length -= pivot; - n -= pivot; - } - } -} - -} // namespace median_of_ninthers_detail - -template -inline void median_of_ninthers_select(Iter begin, Iter mid, Iter end, - Compare comp) { - if (mid == end) return; - using CompType = typename median_common_detail::CompareRefType::type; - - median_of_ninthers_detail::adaptiveQuickselect( - begin, mid - begin, end - begin, comp); -} - -template -inline void median_of_ninthers_select(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_ninthers_select(begin, mid, end, std::less()); -} - -template -inline void median_of_ninthers_sort(Iter begin, Iter mid, Iter end, - Compare comp) { - if (begin == mid) return; - using CompType = typename median_common_detail::CompareRefType::type; - - median_of_ninthers_detail::adaptiveQuickselect( - begin, mid - begin - 1, end - begin, comp); - std::sort(begin, mid, comp); -} - -template -inline void median_of_ninthers_sort(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - median_of_ninthers_sort(begin, mid, end, std::less()); -} - -} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/pdqselect.h b/contrib/miniselect/include/miniselect/pdqselect.h deleted file mode 100644 index 0a22d059103..00000000000 --- a/contrib/miniselect/include/miniselect/pdqselect.h +++ /dev/null @@ -1,935 +0,0 @@ -/* - pdqsort.h - Pattern-defeating quicksort. - - Copyright (c) 2015 Orson Peters - - This software is provided 'as-is', without any express or implied warranty. - In no event will the authors be held liable for any damages arising from the - use of this software. - - Permission is granted to anyone to use this software for any purpose, - including commercial applications, and to alter it and redistribute it - freely, subject to the following restrictions: - - 1. The origin of this software must not be misrepresented; you must not - claim that you wrote the original software. If you use this software in a - product, an acknowledgment in the product documentation would be appreciated - but is not required. - - 2. Altered source versions must be plainly marked as such, and must not be - misrepresented as being the original software. - - 3. This notice may not be removed or altered from any source distribution. -*/ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -// Adjusted by Danila Kutenin to support pdqselect and pdqpartial_sort. - -#ifndef PDQSORT_H -#define PDQSORT_H - -#include -#include -#include -#include -#include - -#if __cplusplus >= 201103L -#include -#include -#define PDQSORT_PREFER_MOVE(x) std::move(x) -#else -#define PDQSORT_PREFER_MOVE(x) (x) -#endif - -namespace miniselect { -namespace pdqsort_detail { - -template -struct CompareRefType { - // Pass the comparator by lvalue reference. Or in debug mode, using a - // debugging wrapper that stores a reference. - using type = typename std::add_lvalue_reference::type; -}; - -enum { - // Partitions below this size are sorted using insertion sort. - insertion_sort_threshold = 24, - - // Partitions above this size use Tukey's ninther to select the pivot. - ninther_threshold = 128, - - // When we detect an already sorted partition, attempt an insertion sort that - // allows this - // amount of element moves before giving up. - partial_insertion_sort_limit = 8, - - // Must be multiple of 8 due to loop unrolling, and < 256 to fit in unsigned - // char. - block_size = 64, - - // Cacheline size, assumes power of two. - cacheline_size = 64 - -}; - -#if __cplusplus >= 201103L -template -struct is_default_compare : std::false_type {}; -template -struct is_default_compare> : std::true_type {}; -template -struct is_default_compare> : std::true_type {}; -#endif - -// Returns floor(log2(n)), assumes n > 0. -template -inline int log2(T n) { - int log = 0; - while (n >>= 1) ++log; - return log; -} - -// Sorts [begin, end) using insertion sort with the given comparison function. -template -inline void insertion_sort(Iter begin, Iter end, Compare& comp) { - typedef typename std::iterator_traits::value_type T; - if (begin == end) return; - - for (Iter cur = begin + 1; cur != end; ++cur) { - Iter sift = cur; - Iter sift_1 = cur - 1; - - // Compare first so we can avoid 2 moves for an element already positioned - // correctly. - if (comp(*sift, *sift_1)) { - T tmp = PDQSORT_PREFER_MOVE(*sift); - - do { - *sift-- = PDQSORT_PREFER_MOVE(*sift_1); - } while (sift != begin && comp(tmp, *--sift_1)); - - *sift = PDQSORT_PREFER_MOVE(tmp); - } - } -} - -// Sorts [begin, end) using insertion sort with the given comparison function. -// Assumes -// *(begin - 1) is an element smaller than or equal to any element in [begin, -// end). -template -inline void unguarded_insertion_sort(Iter begin, Iter end, Compare& comp) { - typedef typename std::iterator_traits::value_type T; - if (begin == end) return; - - for (Iter cur = begin + 1; cur != end; ++cur) { - Iter sift = cur; - Iter sift_1 = cur - 1; - - // Compare first so we can avoid 2 moves for an element already positioned - // correctly. - if (comp(*sift, *sift_1)) { - T tmp = PDQSORT_PREFER_MOVE(*sift); - - do { - *sift-- = PDQSORT_PREFER_MOVE(*sift_1); - } while (comp(tmp, *--sift_1)); - - *sift = PDQSORT_PREFER_MOVE(tmp); - } - } -} - -// Attempts to use insertion sort on [begin, end). Will return false if more -// than partial_insertion_sort_limit elements were moved, and abort sorting. -// Otherwise it will successfully sort and return true. -template -inline bool partial_insertion_sort(Iter begin, Iter end, Compare& comp) { - typedef typename std::iterator_traits::value_type T; - if (begin == end) return true; - - std::size_t limit = 0; - for (Iter cur = begin + 1; cur != end; ++cur) { - Iter sift = cur; - Iter sift_1 = cur - 1; - - // Compare first so we can avoid 2 moves for an element already positioned - // correctly. - if (comp(*sift, *sift_1)) { - T tmp = PDQSORT_PREFER_MOVE(*sift); - - do { - *sift-- = PDQSORT_PREFER_MOVE(*sift_1); - } while (sift != begin && comp(tmp, *--sift_1)); - - *sift = PDQSORT_PREFER_MOVE(tmp); - limit += cur - sift; - } - - if (limit > partial_insertion_sort_limit) return false; - } - - return true; -} - -template -inline void sort2(Iter a, Iter b, Compare& comp) { - if (comp(*b, *a)) std::iter_swap(a, b); -} - -// Sorts the elements *a, *b and *c using comparison function comp. -template -inline void sort3(Iter a, Iter b, Iter c, Compare& comp) { - sort2(a, b, comp); - sort2(b, c, comp); - sort2(a, b, comp); -} - -template -inline T* align_cacheline(T* p) { -#if defined(UINTPTR_MAX) && __cplusplus >= 201103L - std::uintptr_t ip = reinterpret_cast(p); -#else - std::size_t ip = reinterpret_cast(p); -#endif - ip = (ip + cacheline_size - 1) & -cacheline_size; - return reinterpret_cast(ip); -} - -template -inline void swap_offsets(Iter first, Iter last, unsigned char* offsets_l, - unsigned char* offsets_r, int num, bool use_swaps) { - typedef typename std::iterator_traits::value_type T; - if (use_swaps) { - // This case is needed for the descending distribution, where we need - // to have proper swapping for pdqsort to remain O(n). - for (int i = 0; i < num; ++i) { - std::iter_swap(first + offsets_l[i], last - offsets_r[i]); - } - } else if (num > 0) { - Iter l = first + offsets_l[0]; - Iter r = last - offsets_r[0]; - T tmp(PDQSORT_PREFER_MOVE(*l)); - *l = PDQSORT_PREFER_MOVE(*r); - for (int i = 1; i < num; ++i) { - l = first + offsets_l[i]; - *r = PDQSORT_PREFER_MOVE(*l); - r = last - offsets_r[i]; - *l = PDQSORT_PREFER_MOVE(*r); - } - *r = PDQSORT_PREFER_MOVE(tmp); - } -} - -// Partitions [begin, end) around pivot *begin using comparison function comp. -// Elements equal to the pivot are put in the right-hand partition. Returns the -// position of the pivot after partitioning and whether the passed sequence -// already was correctly partitioned. Assumes the pivot is a median of at least -// 3 elements and that [begin, end) is at least insertion_sort_threshold long. -// Uses branchless partitioning. -template -inline std::pair partition_right_branchless(Iter begin, Iter end, - Compare& comp) { - typedef typename std::iterator_traits::value_type T; - - // Move pivot into local for speed. - T pivot(PDQSORT_PREFER_MOVE(*begin)); - Iter first = begin; - Iter last = end; - - // Find the first element greater than or equal than the pivot (the median of - // 3 guarantees this exists). - while (comp(*++first, pivot)) - ; - - // Find the first element strictly smaller than the pivot. We have to guard - // this search if there was no element before *first. - if (first - 1 == begin) - while (first < last && !comp(*--last, pivot)) - ; - else - while (!comp(*--last, pivot)) - ; - - // If the first pair of elements that should be swapped to partition are the - // same element, the passed in sequence already was correctly partitioned. - bool already_partitioned = first >= last; - if (!already_partitioned) { - std::iter_swap(first, last); - ++first; - } - - // The following branchless partitioning is derived from "BlockQuicksort: How - // Branch Mispredictions don’t affect Quicksort" by Stefan Edelkamp and Armin - // Weiss. - unsigned char offsets_l_storage[block_size + cacheline_size]; - unsigned char offsets_r_storage[block_size + cacheline_size]; - unsigned char* offsets_l = align_cacheline(offsets_l_storage); - unsigned char* offsets_r = align_cacheline(offsets_r_storage); - int num_l, num_r, start_l, start_r; - num_l = num_r = start_l = start_r = 0; - - while (last - first > 2 * block_size) { - // Fill up offset blocks with elements that are on the wrong side. - if (num_l == 0) { - start_l = 0; - Iter it = first; - for (unsigned char i = 0; i < block_size;) { - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - } - } - if (num_r == 0) { - start_r = 0; - Iter it = last; - for (unsigned char i = 0; i < block_size;) { - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - } - } - - // Swap elements and update block sizes and first/last boundaries. - int num = std::min(num_l, num_r); - swap_offsets(first, last, offsets_l + start_l, offsets_r + start_r, num, - num_l == num_r); - num_l -= num; - num_r -= num; - start_l += num; - start_r += num; - if (num_l == 0) first += block_size; - if (num_r == 0) last -= block_size; - } - - int l_size = 0, r_size = 0; - int unknown_left = (int)(last - first) - ((num_r || num_l) ? block_size : 0); - if (num_r) { - // Handle leftover block by assigning the unknown elements to the other - // block. - l_size = unknown_left; - r_size = block_size; - } else if (num_l) { - l_size = block_size; - r_size = unknown_left; - } else { - // No leftover block, split the unknown elements in two blocks. - l_size = unknown_left / 2; - r_size = unknown_left - l_size; - } - - // Fill offset buffers if needed. - if (unknown_left && !num_l) { - start_l = 0; - Iter it = first; - for (unsigned char i = 0; i < l_size;) { - offsets_l[num_l] = i++; - num_l += !comp(*it, pivot); - ++it; - } - } - if (unknown_left && !num_r) { - start_r = 0; - Iter it = last; - for (unsigned char i = 0; i < r_size;) { - offsets_r[num_r] = ++i; - num_r += comp(*--it, pivot); - } - } - - int num = std::min(num_l, num_r); - swap_offsets(first, last, offsets_l + start_l, offsets_r + start_r, num, - num_l == num_r); - num_l -= num; - num_r -= num; - start_l += num; - start_r += num; - if (num_l == 0) first += l_size; - if (num_r == 0) last -= r_size; - - // We have now fully identified [first, last)'s proper position. Swap the last - // elements. - if (num_l) { - offsets_l += start_l; - while (num_l--) std::iter_swap(first + offsets_l[num_l], --last); - first = last; - } - if (num_r) { - offsets_r += start_r; - while (num_r--) std::iter_swap(last - offsets_r[num_r], first), ++first; - last = first; - } - - // Put the pivot in the right place. - Iter pivot_pos = first - 1; - *begin = PDQSORT_PREFER_MOVE(*pivot_pos); - *pivot_pos = PDQSORT_PREFER_MOVE(pivot); - - return std::make_pair(pivot_pos, already_partitioned); -} - -// Partitions [begin, end) around pivot *begin using comparison function comp. -// Elements equal to the pivot are put in the right-hand partition. Returns the -// position of the pivot after partitioning and whether the passed sequence -// already was correctly partitioned. Assumes the pivot is a median of at least -// 3 elements and that [begin, end) is at least insertion_sort_threshold long. -template -inline std::pair partition_right(Iter begin, Iter end, - Compare& comp) { - typedef typename std::iterator_traits::value_type T; - - // Move pivot into local for speed. - T pivot(PDQSORT_PREFER_MOVE(*begin)); - - Iter first = begin; - Iter last = end; - - // Find the first element greater than or equal than the pivot (the median of - // 3 guarantees this exists). - while (comp(*++first, pivot)) - ; - - // Find the first element strictly smaller than the pivot. We have to guard - // this search if there was no element before *first. - if (first - 1 == begin) - while (first < last && !comp(*--last, pivot)) - ; - else - while (!comp(*--last, pivot)) - ; - - // If the first pair of elements that should be swapped to partition are the - // same element, the passed in sequence already was correctly partitioned. - bool already_partitioned = first >= last; - - // Keep swapping pairs of elements that are on the wrong side of the pivot. - // Previously swapped pairs guard the searches, which is why the first - // iteration is special-cased above. - while (first < last) { - std::iter_swap(first, last); - while (comp(*++first, pivot)) - ; - while (!comp(*--last, pivot)) - ; - } - - // Put the pivot in the right place. - Iter pivot_pos = first - 1; - *begin = PDQSORT_PREFER_MOVE(*pivot_pos); - *pivot_pos = PDQSORT_PREFER_MOVE(pivot); - - return std::make_pair(pivot_pos, already_partitioned); -} - -// Similar function to the one above, except elements equal to the pivot are put -// to the left of the pivot and it doesn't check or return if the passed -// sequence already was partitioned. Since this is rarely used (the many equal -// case), and in that case pdqsort already has O(n) performance, no block -// quicksort is applied here for simplicity. -template -inline Iter partition_left(Iter begin, Iter end, Compare& comp) { - typedef typename std::iterator_traits::value_type T; - - T pivot(PDQSORT_PREFER_MOVE(*begin)); - Iter first = begin; - Iter last = end; - - while (comp(pivot, *--last)) - ; - - if (last + 1 == end) - while (first < last && !comp(pivot, *++first)) - ; - else - while (!comp(pivot, *++first)) - ; - - while (first < last) { - std::iter_swap(first, last); - while (comp(pivot, *--last)) - ; - while (!comp(pivot, *++first)) - ; - } - - Iter pivot_pos = last; - *begin = PDQSORT_PREFER_MOVE(*pivot_pos); - *pivot_pos = PDQSORT_PREFER_MOVE(pivot); - - return pivot_pos; -} - -template -inline void pdqsort_loop(Iter begin, Iter end, Compare& comp, int bad_allowed, - bool leftmost = true) { - typedef typename std::iterator_traits::difference_type diff_t; - - // Use a while loop for tail recursion elimination. - while (true) { - diff_t size = end - begin; - - // Insertion sort is faster for small arrays. - if (size < insertion_sort_threshold) { - if (leftmost) - insertion_sort(begin, end, comp); - else - unguarded_insertion_sort(begin, end, comp); - return; - } - - // Choose pivot as median of 3 or pseudomedian of 9. - diff_t s2 = size / 2; - if (size > ninther_threshold) { - sort3(begin, begin + s2, end - 1, comp); - sort3(begin + 1, begin + (s2 - 1), end - 2, comp); - sort3(begin + 2, begin + (s2 + 1), end - 3, comp); - sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); - std::iter_swap(begin, begin + s2); - } else - sort3(begin + s2, begin, end - 1, comp); - - // If *(begin - 1) is the end of the right partition of a previous partition - // operation there is no element in [begin, end) that is smaller than - // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change - // strategy, putting equal elements in the left partition, greater elements - // in the right partition. We do not have to recurse on the left partition, - // since it's sorted (all equal). - if (!leftmost && !comp(*(begin - 1), *begin)) { - begin = partition_left(begin, end, comp) + 1; - continue; - } - - // Partition and get results. - std::pair part_result = - Branchless ? partition_right_branchless(begin, end, comp) - : partition_right(begin, end, comp); - Iter pivot_pos = part_result.first; - bool already_partitioned = part_result.second; - - // Check for a highly unbalanced partition. - diff_t l_size = pivot_pos - begin; - diff_t r_size = end - (pivot_pos + 1); - bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; - - // If we got a highly unbalanced partition we shuffle elements to break many - // patterns. - if (highly_unbalanced) { - // If we had too many bad partitions, switch to heapsort to guarantee O(n - // log n). - if (--bad_allowed == 0) { - std::make_heap(begin, end, comp); - std::sort_heap(begin, end, comp); - return; - } - - if (l_size >= insertion_sort_threshold) { - std::iter_swap(begin, begin + l_size / 4); - std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); - - if (l_size > ninther_threshold) { - std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); - std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); - std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); - std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); - } - } - - if (r_size >= insertion_sort_threshold) { - std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); - std::iter_swap(end - 1, end - r_size / 4); - - if (r_size > ninther_threshold) { - std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); - std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); - std::iter_swap(end - 2, end - (1 + r_size / 4)); - std::iter_swap(end - 3, end - (2 + r_size / 4)); - } - } - } else { - // If we were decently balanced and we tried to sort an already - // partitioned sequence try to use insertion sort. - if (already_partitioned && - partial_insertion_sort(begin, pivot_pos, comp) && - partial_insertion_sort(pivot_pos + 1, end, comp)) - return; - } - - // Sort the left partition first using recursion and do tail recursion - // elimination for the right-hand partition. - pdqsort_loop(begin, pivot_pos, comp, bad_allowed, - leftmost); - begin = pivot_pos + 1; - leftmost = false; - } -} - -template -inline void pdqpartial_sort_loop(Iter begin, Iter mid, Iter end, Compare& comp, - int bad_allowed, bool leftmost = true) { - typedef typename std::iterator_traits::difference_type diff_t; - - // Use a while loop for tail recursion elimination. - while (true) { - diff_t size = end - begin; - - // Insertion sort is faster for small arrays. - if (size < insertion_sort_threshold) { - if (leftmost) - insertion_sort(begin, end, comp); - else - unguarded_insertion_sort(begin, end, comp); - return; - } - - // Choose pivot as median of 3 or pseudomedian of 9. - diff_t s2 = size / 2; - if (size > ninther_threshold) { - sort3(begin, begin + s2, end - 1, comp); - sort3(begin + 1, begin + (s2 - 1), end - 2, comp); - sort3(begin + 2, begin + (s2 + 1), end - 3, comp); - sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); - std::iter_swap(begin, begin + s2); - } else - sort3(begin + s2, begin, end - 1, comp); - - // If *(begin - 1) is the end of the right partition of a previous partition - // operation there is no element in [begin, end) that is smaller than - // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change - // strategy, putting equal elements in the left partition, greater elements - // in the right partition. We do not have to recurse on the left partition, - // since it's sorted (all equal). - if (!leftmost && !comp(*(begin - 1), *begin)) { - begin = partition_left(begin, end, comp) + 1; - continue; - } - - // Partition and get results. - std::pair part_result = - Branchless ? partition_right_branchless(begin, end, comp) - : partition_right(begin, end, comp); - Iter pivot_pos = part_result.first; - bool already_partitioned = part_result.second; - - // Check for a highly unbalanced partition. - diff_t l_size = pivot_pos - begin; - diff_t r_size = end - (pivot_pos + 1); - bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; - - // If we got a highly unbalanced partition we shuffle elements to break many - // patterns. - if (highly_unbalanced) { - // If we had too many bad partitions, switch to heapsort to guarantee O(n - // log n). - if (--bad_allowed == 0) { - std::make_heap(begin, end, comp); - std::sort_heap(begin, end, comp); - return; - } - - if (l_size >= insertion_sort_threshold) { - std::iter_swap(begin, begin + l_size / 4); - std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); - - if (l_size > ninther_threshold) { - std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); - std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); - std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); - std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); - } - } - - if (r_size >= insertion_sort_threshold) { - std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); - std::iter_swap(end - 1, end - r_size / 4); - - if (r_size > ninther_threshold) { - std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); - std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); - std::iter_swap(end - 2, end - (1 + r_size / 4)); - std::iter_swap(end - 3, end - (2 + r_size / 4)); - } - } - } else { - // If we were decently balanced and we tried to sort an already - // partitioned sequence try to use insertion sort. - if (already_partitioned && - partial_insertion_sort(begin, pivot_pos, comp) && - partial_insertion_sort(pivot_pos + 1, end, comp)) - return; - } - - // Sort the left partition first using recursion and do tail recursion - // elimination for the right-hand partition. - if (pivot_pos < mid) { - pdqsort_loop(begin, pivot_pos, comp, - bad_allowed, leftmost); - begin = pivot_pos + 1; - leftmost = false; - } else { - end = pivot_pos; - } - } -} - -template -inline void pdqselect_loop(Iter begin, Iter mid, Iter end, Compare& comp, - int bad_allowed, bool leftmost = true) { - typedef typename std::iterator_traits::difference_type diff_t; - - // Use a while loop for tail recursion elimination. - while (true) { - diff_t size = end - begin; - - // Insertion sort is faster for small arrays. - if (size < insertion_sort_threshold) { - if (leftmost) - insertion_sort(begin, end, comp); - else - unguarded_insertion_sort(begin, end, comp); - return; - } - - // Choose pivot as median of 3 or pseudomedian of 9. - diff_t s2 = size / 2; - if (size > ninther_threshold) { - sort3(begin, begin + s2, end - 1, comp); - sort3(begin + 1, begin + (s2 - 1), end - 2, comp); - sort3(begin + 2, begin + (s2 + 1), end - 3, comp); - sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); - std::iter_swap(begin, begin + s2); - } else - sort3(begin + s2, begin, end - 1, comp); - - // If *(begin - 1) is the end of the right partition of a previous partition - // operation there is no element in [begin, end) that is smaller than - // *(begin - 1). Then if our pivot compares equal to *(begin - 1) we change - // strategy, putting equal elements in the left partition, greater elements - // in the right partition. We do not have to recurse on the left partition, - // since it's sorted (all equal). - if (!leftmost && !comp(*(begin - 1), *begin)) { - begin = partition_left(begin, end, comp) + 1; - continue; - } - - // Partition and get results. - std::pair part_result = - Branchless ? partition_right_branchless(begin, end, comp) - : partition_right(begin, end, comp); - Iter pivot_pos = part_result.first; - bool already_partitioned = part_result.second; - - // Check for a highly unbalanced partition. - diff_t l_size = pivot_pos - begin; - diff_t r_size = end - (pivot_pos + 1); - bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; - - // If we got a highly unbalanced partition we shuffle elements to break many - // patterns. - if (highly_unbalanced) { - // If we had too many bad partitions, switch to heapsort to guarantee O(n - // log n). - if (--bad_allowed == 0) { - std::nth_element(begin, mid, end, comp); - return; - } - - if (l_size >= insertion_sort_threshold) { - std::iter_swap(begin, begin + l_size / 4); - std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); - - if (l_size > ninther_threshold) { - std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); - std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); - std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); - std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); - } - } - - if (r_size >= insertion_sort_threshold) { - std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); - std::iter_swap(end - 1, end - r_size / 4); - - if (r_size > ninther_threshold) { - std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); - std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); - std::iter_swap(end - 2, end - (1 + r_size / 4)); - std::iter_swap(end - 3, end - (2 + r_size / 4)); - } - } - } else { - // If we were decently balanced and we tried to sort an already - // partitioned sequence try to use insertion sort. - if (already_partitioned && - partial_insertion_sort(begin, pivot_pos, comp) && - partial_insertion_sort(pivot_pos + 1, end, comp)) - return; - } - // Sort the left partition first using recursion and do tail recursion - // elimination for the right-hand partition. - if (pivot_pos < mid) { - begin = pivot_pos + 1; - leftmost = false; - } else { - end = pivot_pos; - } - } -} -} // namespace pdqsort_detail - -template -inline void pdqsort(Iter begin, Iter end, Compare comp) { - if (begin == end) return; - -#if __cplusplus >= 201103L - pdqsort_detail::pdqsort_loop< - Iter, Compare, - pdqsort_detail::is_default_compare< - typename std::decay::type>::value && - std::is_arithmetic< - typename std::iterator_traits::value_type>::value>( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#else - pdqsort_detail::pdqsort_loop( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#endif -} - -template -inline void pdqsort(Iter begin, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqsort(begin, end, std::less()); -} - -template -inline void pdqsort_branchless(Iter begin, Iter end, Compare comp) { - if (begin == end) return; - pdqsort_detail::pdqsort_loop( - begin, end, comp, pdqsort_detail::log2(end - begin)); -} - -template -inline void pdqsort_branchless(Iter begin, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqsort_branchless(begin, end, std::less()); -} - -template -inline void pdqpartial_sort(Iter begin, Iter mid, Iter end, Compare comp) { - if (begin == end) return; - -#if __cplusplus >= 201103L - pdqsort_detail::pdqpartial_sort_loop< - Iter, Compare, - pdqsort_detail::is_default_compare< - typename std::decay::type>::value && - std::is_arithmetic< - typename std::iterator_traits::value_type>::value>( - begin, mid, end, comp, pdqsort_detail::log2(end - begin)); -#else - pdqsort_detail::pdqpartial_sort_loop( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#endif -} - -template -inline void pdqpartial_sort(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqpartial_sort(begin, mid, end, std::less()); -} - -template -inline void pdqpartial_sort_branchless(Iter begin, Iter mid, Iter end, - Compare comp) { - if (begin == end) return; - pdqsort_detail::pdqpartial_sort_loop( - begin, mid, end, comp, pdqsort_detail::log2(end - begin)); -} - -template -inline void pdqpartial_sort_branchless(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqpartial_sort_branchless(begin, mid, end, std::less()); -} - -template -inline void pdqselect(Iter begin, Iter mid, Iter end, Compare comp) { - if (mid == end) return; - using CompType = typename median_common_detail::CompareRefType::type; - -#if __cplusplus >= 201103L - pdqsort_detail::pdqselect_loop< - Iter, CompType, - pdqsort_detail::is_default_compare< - typename std::decay::type>::value && - std::is_arithmetic< - typename std::iterator_traits::value_type>::value>( - begin, mid, end, comp, pdqsort_detail::log2(end - begin)); -#else - pdqsort_detail::pdqselect_loop( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#endif -} - -template -inline void pdqselect(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqselect(begin, mid, end, std::less()); -} - -template -inline void pdqselect_branchless(Iter begin, Iter mid, Iter end, Compare comp) { - if (mid == end) return; - using CompType = typename median_common_detail::CompareRefType::type; - pdqsort_detail::pdqselect_loop( - begin, mid, end, comp, pdqsort_detail::log2(end - begin)); -} - -template -inline void pdqselect_branchless(Iter begin, Iter mid, Iter end) { - typedef typename std::iterator_traits::value_type T; - pdqselect_branchless(begin, mid, end, std::less()); -} - -#undef PDQSORT_PREFER_MOVE - -#endif - -} // namespace miniselect diff --git a/contrib/miniselect/include/miniselect/private/median_common.h b/contrib/miniselect/include/miniselect/private/median_common.h deleted file mode 100644 index 30cb1323bbf..00000000000 --- a/contrib/miniselect/include/miniselect/private/median_common.h +++ /dev/null @@ -1,437 +0,0 @@ -/* Copyright Andrei Alexandrescu, 2016-, - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include -#include -#include - -namespace miniselect { -namespace median_common_detail { - -template -struct CompareRefType { - // Pass the comparator by lvalue reference. Or in debug mode, using a - // debugging wrapper that stores a reference. - using type = typename std::add_lvalue_reference::type; -}; -/** -Swaps the median of r[a], r[b], and r[c] into r[b]. -*/ -template -void median3(Iter r, size_t a, size_t b, size_t c, Compare&& comp) { - if (comp(r[b], r[a])) // b < a - { - if (comp(r[b], r[c])) // b < a, b < c - { - if (comp(r[c], r[a])) // b < c < a - std::swap(r[b], r[c]); - else // b < a <= c - std::swap(r[b], r[a]); - } - } else if (comp(r[c], r[b])) // a <= b, c < b - { - if (comp(r[c], r[a])) // c < a <= b - std::swap(r[b], r[a]); - else // a <= c < b - std::swap(r[b], r[c]); - } -} - -/** -Sorts in place r[a], r[b], and r[c]. -*/ -template -void sort3(Iter r, size_t a, size_t b, size_t c, Compare&& comp) { - typedef typename std::iterator_traits::value_type T; - if (comp(r[b], r[a])) // b < a - { - if (comp(r[c], r[b])) // c < b < a - { - std::swap(r[a], r[c]); // a < b < c - } else // b < a, b <= c - { - T t = std::move(r[a]); - r[a] = std::move(r[b]); - if (comp(r[c], t)) // b <= c < a - { - r[b] = std::move(r[c]); - r[c] = std::move(t); - } else // b < a <= c - { - r[b] = std::move(t); - } - } - } else if (comp(r[c], r[b])) // a <= b, c < b - { - T t = std::move(r[c]); - r[c] = std::move(r[b]); - if (comp(t, r[a])) // c < a < b - { - r[b] = std::move(r[a]); - r[a] = std::move(t); - } else // a <= c < b - { - r[b] = std::move(t); - } - } - - assert(!comp(r[b], r[a]) && !comp(r[c], r[b])); -} - -/** -If leanRight == false, swaps the lower median of r[a]...r[d] into r[b] and -the minimum into r[a]. If leanRight == true, swaps the upper median of -r[a]...r[d] into r[c] and the minimum into r[d]. -*/ -template -void partition4(Iter r, size_t a, size_t b, size_t c, size_t d, - Compare&& comp) { - assert(a != b && a != c && a != d && b != c && b != d && c != d); - /* static */ if (leanRight) { - // In the median of 5 algorithm, consider r[e] infinite - if (comp(r[c], r[a])) { - std::swap(r[a], r[c]); - } // a <= c - if (comp(r[d], r[b])) { - std::swap(r[b], r[d]); - } // a <= c, b <= d - if (comp(r[d], r[c])) { - std::swap(r[c], r[d]); // a <= d, b <= c < d - std::swap(r[a], r[b]); // b <= d, a <= c < d - } // a <= c <= d, b <= d - if (comp(r[c], r[b])) { // a <= c <= d, c < b <= d - std::swap(r[b], r[c]); // a <= b <= c <= d - } // a <= b <= c <= d - } else { - // In the median of 5 algorithm consider r[a] infinitely small, then - // change b->a. c->b, d->c, e->d - if (comp(r[c], r[a])) { - std::swap(r[a], r[c]); - } - if (comp(r[c], r[b])) { - std::swap(r[b], r[c]); - } - if (comp(r[d], r[a])) { - std::swap(r[a], r[d]); - } - if (comp(r[d], r[b])) { - std::swap(r[b], r[d]); - } else { - if (comp(r[b], r[a])) { - std::swap(r[a], r[b]); - } - } - } -} - -/** -Places the median of r[a]...r[e] in r[c] and partitions the other elements -around it. -*/ -template -void partition5(Iter r, size_t a, size_t b, size_t c, size_t d, size_t e, - Compare&& comp) { - assert(a != b && a != c && a != d && a != e && b != c && b != d && b != e && - c != d && c != e && d != e); - if (comp(r[c], r[a])) { - std::swap(r[a], r[c]); - } - if (comp(r[d], r[b])) { - std::swap(r[b], r[d]); - } - if (comp(r[d], r[c])) { - std::swap(r[c], r[d]); - std::swap(r[a], r[b]); - } - if (comp(r[e], r[b])) { - std::swap(r[b], r[e]); - } - if (comp(r[e], r[c])) { - std::swap(r[c], r[e]); - if (comp(r[c], r[a])) { - std::swap(r[a], r[c]); - } - } else { - if (comp(r[c], r[b])) { - std::swap(r[b], r[c]); - } - } -} - -/** -Implements Hoare partition. -*/ -template -Iter pivotPartition(Iter r, size_t k, size_t length, Compare&& comp) { - assert(k < length); - std::swap(*r, r[k]); - size_t lo = 1, hi = length - 1; - for (;; ++lo, --hi) { - for (;; ++lo) { - if (lo > hi) goto loop_done; - if (!comp(r[lo], *r)) break; - } - // found the left bound: r[lo] >= r[0] - assert(lo <= hi); - for (; comp(*r, r[hi]); --hi) { - } - if (lo >= hi) break; - // found the right bound: r[hi] <= r[0], swap & make progress - std::swap(r[lo], r[hi]); - } -loop_done: - --lo; - std::swap(r[lo], *r); - return r + lo; -} - -/** -Implements the quickselect algorithm, parameterized with a partition function. -*/ -template -void quickselect(Iter r, Iter mid, Iter end, Compare&& comp) { - if (r == end || mid >= end) return; - assert(r <= mid && mid < end); - for (;;) switch (end - r) { - case 1: - return; - case 2: - if (comp(r[1], *r)) std::swap(*r, r[1]); - return; - case 3: - sort3(r, 0, 1, 2, comp); - return; - case 4: - switch (mid - r) { - case 0: - goto select_min; - case 1: - partition4(r, 0, 1, 2, 3, comp); - break; - case 2: - partition4(r, 0, 1, 2, 3, comp); - break; - case 3: - goto select_max; - default: - assert(false); - } - return; - default: - assert(end - r > 4); - if (r == mid) { - select_min: - auto pivot = r; - for (++mid; mid < end; ++mid) - if (comp(*mid, *pivot)) pivot = mid; - std::swap(*r, *pivot); - return; - } - if (mid + 1 == end) { - select_max: - auto pivot = r; - for (mid = r + 1; mid < end; ++mid) - if (comp(*pivot, *mid)) pivot = mid; - std::swap(*pivot, end[-1]); - return; - } - auto pivot = partition(r, end, comp); - if (pivot == mid) return; - if (mid < pivot) { - end = pivot; - } else { - r = pivot + 1; - } - } -} - -/** -Returns the index of the median of r[a], r[b], and r[c] without writing -anything. -*/ -template -size_t medianIndex(const Iter r, size_t a, size_t b, size_t c, Compare&& comp) { - if (r[a] > r[c]) std::swap(a, c); - if (r[b] > r[c]) return c; - if (comp(r[b], r[a])) return a; - return b; -} - -/** -Returns the index of the median of r[a], r[b], r[c], and r[d] without writing -anything. If leanRight is true, computes the upper median. Otherwise, conputes -the lower median. -*/ -template -static size_t medianIndex(Iter r, size_t a, size_t b, size_t c, size_t d, - Compare&& comp) { - if (comp(r[d], r[c])) std::swap(c, d); - assert(r[c] <= r[d]); - /* static */ if (leanRight) { - if (comp(r[c], r[a])) { - assert(comp(r[c], r[a]) && !comp(r[d], r[c])); // so r[c]) is out - return medianIndex(r, a, b, d, comp); - } - } else { - if (!comp(r[d], r[a])) { - return medianIndex(r, a, b, c, comp); - } - } - // Could return medianIndex(r, b, c, d) but we already know r[c] <= r[d] - if (!comp(r[c], r[b])) return c; - if (comp(r[d], r[b])) return d; - return b; -} - -/** -Tukey's Ninther: compute the median of r[_1], r[_2], r[_3], then the median of -r[_4], r[_5], r[_6], then the median of r[_7], r[_8], r[_9], and then swap the -median of those three medians into r[_5]. -*/ -template -void ninther(Iter r, size_t _1, size_t _2, size_t _3, size_t _4, size_t _5, - size_t _6, size_t _7, size_t _8, size_t _9, Compare&& comp) { - _2 = medianIndex(r, _1, _2, _3, comp); - _8 = medianIndex(r, _7, _8, _9, comp); - if (comp(r[_8], r[_2])) std::swap(_2, _8); - if (comp(r[_6], r[_4])) std::swap(_4, _6); - // Here we know that r[_2] and r[_8] are the other two medians and that - // r[_2] <= r[_8]. We also know that r[_4] <= r[_6] - if (comp(r[_5], r[_4])) { - // r[_4] is the median of r[_4], r[_5], r[_6] - } else if (comp(r[_6], r[_5])) { - // r[_6] is the median of r[_4], r[_5], r[_6] - _4 = _6; - } else { - // Here we know r[_5] is the median of r[_4], r[_5], r[_6] - if (comp(r[_5], r[_2])) return std::swap(r[_5], r[_2]); - if (comp(r[_8], r[_5])) return std::swap(r[_5], r[_8]); - // This is the only path that returns with no swap - return; - } - // Here we know r[_4] is the median of r[_4], r[_5], r[_6] - if (comp(r[_4], r[_2])) - _4 = _2; - else if (comp(r[_8], r[_4])) - _4 = _8; - std::swap(r[_5], r[_4]); -} - -/** -Input assumptions: -(a) hi <= rite -(c) the range r[0 .. hi] contains elements no smaller than r[0] -Output guarantee: same as Hoare partition using r[0] as pivot. Returns the new -position of the pivot. -*/ -template -size_t expandPartitionRight(Iter r, size_t hi, size_t rite, Compare&& comp) { - size_t pivot = 0; - assert(pivot <= hi); - assert(hi <= rite); - // First loop: spend r[pivot .. hi] - for (; pivot < hi; --rite) { - if (rite == hi) goto done; - if (!comp(r[rite], r[0])) continue; - ++pivot; - std::swap(r[rite], r[pivot]); - } - // Second loop: make left and pivot meet - for (; rite > pivot; --rite) { - if (!comp(r[rite], r[0])) continue; - while (rite > pivot) { - ++pivot; - if (comp(r[0], r[pivot])) { - std::swap(r[rite], r[pivot]); - break; - } - } - } - -done: - std::swap(r[0], r[pivot]); - return pivot; -} - -/** -Input assumptions: -(a) lo > 0, lo <= pivot -(b) the range r[lo .. pivot] already contains elements no greater than r[pivot] -Output guarantee: Same as Hoare partition around r[pivot]. Returns the new -position of the pivot. -*/ -template -size_t expandPartitionLeft(Iter r, size_t lo, size_t pivot, Compare&& comp) { - assert(lo > 0 && lo <= pivot); - size_t left = 0; - const auto oldPivot = pivot; - for (; lo < pivot; ++left) { - if (left == lo) goto done; - if (!comp(r[oldPivot], r[left])) continue; - --pivot; - std::swap(r[left], r[pivot]); - } - // Second loop: make left and pivot meet - for (;; ++left) { - if (left == pivot) break; - if (!comp(r[oldPivot], r[left])) continue; - for (;;) { - if (left == pivot) goto done; - --pivot; - if (comp(r[pivot], r[oldPivot])) { - std::swap(r[left], r[pivot]); - break; - } - } - } - -done: - std::swap(r[oldPivot], r[pivot]); - return pivot; -} - -/** -Input assumptions: -(a) lo <= pivot, pivot < hi, hi <= length -(b) the range r[lo .. pivot] already contains elements no greater than -r[pivot] -(c) the range r[pivot .. hi] already contains elements no smaller than -r[pivot] -Output guarantee: Same as Hoare partition around r[pivot], returning the new -position of the pivot. -*/ -template -size_t expandPartition(Iter r, size_t lo, size_t pivot, size_t hi, - size_t length, Compare&& comp) { - assert(lo <= pivot && pivot < hi && hi <= length); - --hi; - --length; - size_t left = 0; - for (;; ++left, --length) { - for (;; ++left) { - if (left == lo) - return pivot + expandPartitionRight(r + pivot, hi - pivot, - length - pivot, comp); - if (comp(r[pivot], r[left])) break; - } - for (;; --length) { - if (length == hi) - return left + - expandPartitionLeft(r + left, lo - left, pivot - left, comp); - if (!comp(r[pivot], r[length])) break; - } - std::swap(r[left], r[length]); - } -} - -} // namespace median_common_detail -} // namespace miniselect diff --git a/contrib/miniselect/testing/test_common.h b/contrib/miniselect/testing/test_common.h deleted file mode 100644 index df0c179c840..00000000000 --- a/contrib/miniselect/testing/test_common.h +++ /dev/null @@ -1,180 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#pragma once - -#include - -#include - -#include "miniselect/floyd_rivest_select.h" -#include "miniselect/median_of_3_random.h" -#include "miniselect/median_of_medians.h" -#include "miniselect/median_of_ninthers.h" -#include "miniselect/pdqselect.h" - -namespace miniselect { -namespace algorithms { - -struct STD { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - std::partial_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - std::partial_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - std::nth_element(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - std::nth_element(begin, mid, end); - } -}; - -struct PDQ { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - pdqpartial_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - pdqpartial_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - pdqselect(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - pdqselect(begin, mid, end); - } -}; - -struct PDQBranchless { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - pdqpartial_sort_branchless(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - pdqpartial_sort_branchless(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - pdqselect_branchless(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - pdqselect_branchless(begin, mid, end); - } -}; - -struct FloydRivest { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - floyd_rivest_partial_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - floyd_rivest_partial_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - floyd_rivest_select(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - floyd_rivest_select(begin, mid, end); - } -}; - -struct MedianOfNinthers { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_ninthers_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - median_of_ninthers_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_ninthers_select(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - median_of_ninthers_select(begin, mid, end); - } -}; - -struct MedianOfMedians { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_medians_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - median_of_medians_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_medians_select(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - median_of_medians_select(begin, mid, end); - } -}; - -struct MedianOf3Random { - template - static void Sort(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_3_random_sort(begin, mid, end, std::move(comp)); - } - - template - static void Sort(Iter begin, Iter mid, Iter end) { - median_of_3_random_sort(begin, mid, end); - } - - template - static void Select(Iter begin, Iter mid, Iter end, Compare&& comp) { - median_of_3_random_select(begin, mid, end, std::move(comp)); - } - - template - static void Select(Iter begin, Iter mid, Iter end) { - median_of_3_random_select(begin, mid, end); - } -}; - -using All = - ::testing::Types; - -} // namespace algorithms -} // namespace miniselect diff --git a/contrib/miniselect/testing/test_select.cpp b/contrib/miniselect/testing/test_select.cpp deleted file mode 100644 index 9b8e9dce970..00000000000 --- a/contrib/miniselect/testing/test_select.cpp +++ /dev/null @@ -1,231 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#include -#include - -#include -#include -#include -#include -#include - -#include "test_common.h" - -using ::testing::Eq; - -namespace miniselect { -namespace { - -struct IndirectLess { - // Non const comparator with deleted copy. - template - bool operator()(const P &x, const P &y) const { - return *x < *y; - } - IndirectLess(const IndirectLess &) = default; - IndirectLess &operator=(const IndirectLess &) = default; - IndirectLess(IndirectLess &&) = default; - IndirectLess &operator=(IndirectLess &&) = default; -}; - -template -class SelectTest : public ::testing::Test { - public: - using Base = Selector; - - static void TestSelects(size_t N, size_t M) { - ASSERT_NE(N, 0); - ASSERT_GT(N, M); - SCOPED_TRACE(N); - SCOPED_TRACE(M); - std::vector array(N); - for (size_t i = 0; i < N; ++i) { - array[i] = i; - } - auto array_smaller = array; - std::mt19937_64 mersenne_engine; - std::shuffle(array.begin(), array.end(), mersenne_engine); - Selector::Select(array.begin(), array.begin() + M, array.end(), - std::greater()); - EXPECT_EQ(array[M], N - M - 1); - for (size_t i = 0; i < M; ++i) { - EXPECT_GE(array[i], array[M]); - } - for (size_t i = M; i < N; ++i) { - EXPECT_LE(array[i], array[M]); - } - std::shuffle(array_smaller.begin(), array_smaller.end(), mersenne_engine); - Selector::Select(array_smaller.begin(), array_smaller.begin() + M, - array_smaller.end()); - EXPECT_EQ(array_smaller[M], M); - for (size_t i = 0; i < M; ++i) { - EXPECT_LE(array_smaller[i], array_smaller[M]); - } - for (size_t i = M; i < N; ++i) { - EXPECT_GE(array_smaller[i], array_smaller[M]); - } - } - - static void TestSelects(size_t N) { - TestSelects(N, 0); - TestSelects(N, 1); - TestSelects(N, 2); - TestSelects(N, 3); - TestSelects(N, N / 2 - 1); - TestSelects(N, N / 2); - TestSelects(N, N / 2 + 1); - TestSelects(N, N - 2); - TestSelects(N, N - 1); - } - - static void TestManySelects() { - TestSelects(10); - TestSelects(256); - TestSelects(257); - TestSelects(499); - TestSelects(500); - TestSelects(997); - TestSelects(1000); - TestSelects(1000 * 100); - TestSelects(1009); - TestSelects(1009 * 109); - } - - static void TestCustomComparators() { - std::vector> v(1000); - for (int i = 0; static_cast(i) < v.size(); ++i) { - v[i] = std::make_unique(i); - } - Selector::Select(v.begin(), v.begin() + v.size() / 2, v.end(), - IndirectLess{}); - EXPECT_EQ(*v[v.size() / 2], v.size() / 2); - for (size_t i = 0; i < v.size() / 2; ++i) { - ASSERT_NE(v[i], nullptr); - EXPECT_LE(*v[i], v.size() / 2); - } - for (size_t i = v.size() / 2; i < v.size(); ++i) { - ASSERT_NE(v[i], nullptr); - EXPECT_GE(*v[i], v.size() / 2); - } - } - - static void TestRepeat(size_t N, size_t M) { - ASSERT_NE(N, 0); - ASSERT_GT(N, M); - SCOPED_TRACE(N); - SCOPED_TRACE(M); - std::mt19937_64 mersenne_engine(10); - std::vector array(N); - for (size_t i = 0; i < M; ++i) { - array[i] = false; - } - for (size_t i = M; i < N; ++i) { - array[i] = true; - } - std::shuffle(array.begin(), array.end(), mersenne_engine); - Selector::Select(array.begin(), array.begin() + M, array.end()); - EXPECT_EQ(array[M], true); - for (size_t i = 0; i < M; ++i) { - EXPECT_EQ(array[i], false); - } - for (size_t i = M; i < N; ++i) { - EXPECT_EQ(array[i], true); - } - std::shuffle(array.begin(), array.end(), mersenne_engine); - Selector::Select(array.begin(), array.begin() + M / 2, array.end()); - EXPECT_EQ(array[M / 2], false); - for (size_t i = 0; i < M / 2; ++i) { - EXPECT_EQ(array[i], false); - } - std::shuffle(array.begin(), array.end(), mersenne_engine); - Selector::Select(array.begin(), array.begin() + M - 1, array.end()); - EXPECT_EQ(array[M - 1], false); - for (size_t i = 0; i < M - 1; ++i) { - EXPECT_EQ(array[i], false); - } - } - - static void TestRepeats(size_t N) { - TestRepeat(N, 1); - TestRepeat(N, 2); - TestRepeat(N, 3); - TestRepeat(N, N / 2 - 1); - TestRepeat(N, N / 2); - TestRepeat(N, N / 2 + 1); - TestRepeat(N, N - 2); - TestRepeat(N, N - 1); - } - - static void TestManyRepeats() { - TestRepeats(10); - TestRepeats(100); - TestRepeats(257); - TestRepeats(1000); - TestRepeats(100000); - } -}; - -TYPED_TEST_SUITE(SelectTest, algorithms::All); - -TYPED_TEST(SelectTest, TestSmall) { - std::vector v = {"ab", "aaa", "ab"}; - TypeParam::Select(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); - v = {"aba"}; - TypeParam::Select(v.begin(), v.begin(), v.end()); - EXPECT_THAT(v, Eq(std::vector{"aba"})); - v.clear(); - TypeParam::Select(v.begin(), v.end(), v.end()); - EXPECT_TRUE(v.empty()); -} - -TYPED_TEST(SelectTest, TestAnotherSmall) { - std::vector v = {"ab", "ab", "aaa"}; - TypeParam::Select(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); -} - -TYPED_TEST(SelectTest, TestEmptySmall) { - std::vector v = {"", ""}; - TypeParam::Select(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"", ""})); -} - -TYPED_TEST(SelectTest, TestBasic) { TestFixture::TestManySelects(); } - -TYPED_TEST(SelectTest, TestComparators) { - TestFixture::TestCustomComparators(); -} - -TYPED_TEST(SelectTest, TestRepeats) { TestFixture::TestManyRepeats(); } - -TYPED_TEST(SelectTest, TestLast) { - std::vector array(100); - for (size_t i = 0; i < 100; ++i) { - array[i] = i; - } - auto array_smaller = array; - std::mt19937_64 mersenne_engine; - std::shuffle(array.begin(), array.end(), mersenne_engine); - auto copy_array = array; - // Should be no effect. - size_t cmp = 0; - TypeParam::Select(array.begin(), array.end(), array.end(), - [&cmp](const auto &lhs, const auto &rhs) { - ++cmp; - return lhs < rhs; - }); - EXPECT_EQ(cmp, 0); - EXPECT_EQ(copy_array, array); -} - -} // namespace -} // namespace miniselect - -int main(int argc, char **argv) { - ::testing::InitGoogleTest(&argc, argv); - return RUN_ALL_TESTS(); -} diff --git a/contrib/miniselect/testing/test_sort.cpp b/contrib/miniselect/testing/test_sort.cpp deleted file mode 100644 index 19c6ff036fe..00000000000 --- a/contrib/miniselect/testing/test_sort.cpp +++ /dev/null @@ -1,161 +0,0 @@ -/* Copyright Danila Kutenin, 2020-. - * Distributed under the Boost Software License, Version 1.0. - * (See accompanying file LICENSE_1_0.txt or copy at - * https://boost.org/LICENSE_1_0.txt) - */ -#include -#include - -#include -#include -#include -#include -#include - -#include "test_common.h" - -using ::testing::Eq; - -namespace miniselect { -namespace { - -struct IndirectLess { - // Non const comparator with deleted copy. - template - bool operator()(const P &x, const P &y) const { - return *x < *y; - } - IndirectLess(const IndirectLess &) = default; - IndirectLess &operator=(const IndirectLess &) = default; - IndirectLess(IndirectLess &&) = default; - IndirectLess &operator=(IndirectLess &&) = default; -}; - -template -class PartialSortTest : public ::testing::Test { - public: - static void TestSorts(size_t N, size_t M) { - ASSERT_NE(N, 0); - ASSERT_GE(N, M); - SCOPED_TRACE(N); - SCOPED_TRACE(M); - std::vector array(N); - for (size_t i = 0; i < N; ++i) { - array[i] = i; - } - auto array_smaller = array; - std::mt19937_64 mersenne_engine; - std::shuffle(array.begin(), array.end(), mersenne_engine); - Sorter::Sort(array.begin(), array.begin() + M, array.end(), - std::greater()); - for (size_t i = 0; i < M; ++i) { - EXPECT_EQ(array[i], N - i - 1); - } - std::shuffle(array_smaller.begin(), array_smaller.end(), mersenne_engine); - Sorter::Sort(array_smaller.begin(), array_smaller.begin() + M, - array_smaller.end()); - for (size_t i = 0; i < M; ++i) { - EXPECT_EQ(array_smaller[i], i); - } - } - - static void TestSorts(size_t N) { - TestSorts(N, 0); - TestSorts(N, 1); - TestSorts(N, 2); - TestSorts(N, 3); - TestSorts(N, N / 2 - 1); - TestSorts(N, N / 2); - TestSorts(N, N / 2 + 1); - TestSorts(N, N - 2); - TestSorts(N, N - 1); - TestSorts(N, N); - } - - static void TestManySorts() { - TestSorts(10); - TestSorts(256); - TestSorts(257); - TestSorts(499); - TestSorts(500); - TestSorts(997); - TestSorts(1000); - TestSorts(1000 * 100); - TestSorts(1009); - TestSorts(1009 * 109); - } - - static void TestCustomComparators() { - std::vector> v(1000); - for (int i = 0; static_cast(i) < v.size(); ++i) { - v[i] = std::make_unique(i); - } - Sorter::Sort(v.begin(), v.begin() + v.size() / 2, v.end(), IndirectLess{}); - for (int i = 0; static_cast(i) < v.size() / 2; ++i) { - ASSERT_NE(v[i], nullptr); - EXPECT_EQ(*v[i], i); - } - } -}; - -TYPED_TEST_SUITE(PartialSortTest, algorithms::All); - -TYPED_TEST(PartialSortTest, TestSmall) { - std::vector v = {"ab", "aaa", "ab"}; - TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); - v = {"aba"}; - TypeParam::Sort(v.begin(), v.begin(), v.end()); - EXPECT_THAT(v, Eq(std::vector{"aba"})); - v.clear(); - TypeParam::Sort(v.begin(), v.end(), v.end()); - EXPECT_TRUE(v.empty()); -} - -TYPED_TEST(PartialSortTest, TestAnotherSmall) { - std::vector v = {"ab", "ab", "aaa"}; - TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"aaa", "ab", "ab"})); -} - -TYPED_TEST(PartialSortTest, TestEmptySmall) { - std::vector v = {"", ""}; - TypeParam::Sort(v.begin(), v.begin() + 1, v.end()); - EXPECT_THAT(v, Eq(std::vector{"", ""})); -} - -TYPED_TEST(PartialSortTest, TestBasic) { TestFixture::TestManySorts(); } - -TYPED_TEST(PartialSortTest, TestComparators) { - TestFixture::TestCustomComparators(); -} - -// The standard says that the order of other elements is unspecified even if -// nothing should be sorted so it fails for libcxx and PDQ which is Ok. Saving -// this test for a reference. -TYPED_TEST(PartialSortTest, DISABLED_TestEmpty) { - std::vector array(100); - for (size_t i = 0; i < 100; ++i) { - array[i] = i; - } - std::mt19937_64 mersenne_engine; - std::shuffle(array.begin(), array.end(), mersenne_engine); - size_t cmp = 0; - auto copy_array = array; - // Should be no effect. - TypeParam::Sort(array.begin(), array.begin(), array.end(), - [&cmp](const auto &lhs, const auto &rhs) { - ++cmp; - return lhs < rhs; - }); - EXPECT_EQ(cmp, 0); - EXPECT_EQ(copy_array, array); -} - -} // namespace -} // namespace miniselect - -int main(int argc, char **argv) { - ::testing::InitGoogleTest(&argc, argv); - return RUN_ALL_TESTS(); -} diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index 3f5a0907126..e3d1ae5d8b8 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -87,7 +88,7 @@ struct QuantileExact : QuantileExactBase> { size_t n = level < 1 ? level * array.size() : (array.size() - 1); - std::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. + miniselect::floyd_rivest_partial_select(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. return array[n]; } @@ -107,7 +108,7 @@ struct QuantileExact : QuantileExactBase> size_t n = level < 1 ? level * array.size() : (array.size() - 1); - std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); + miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); result[indices[i]] = array[n]; prev_n = n; @@ -144,7 +145,7 @@ struct QuantileExactExclusive : public QuantileExact else if (n < 1) return static_cast(array[0]); - std::nth_element(array.begin(), array.begin() + n - 1, array.end()); + miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); auto nth_element = std::min_element(array.begin() + n, array.end()); return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); @@ -173,7 +174,7 @@ struct QuantileExactExclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { - std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); + miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); auto nth_element = std::min_element(array.begin() + n, array.end()); result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); @@ -209,7 +210,7 @@ struct QuantileExactInclusive : public QuantileExact else if (n < 1) return static_cast(array[0]); - std::nth_element(array.begin(), array.begin() + n - 1, array.end()); + miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); auto nth_element = std::min_element(array.begin() + n, array.end()); return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); @@ -236,7 +237,7 @@ struct QuantileExactInclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { - std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); + miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); auto nth_element = std::min_element(array.begin() + n, array.end()); result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index 2ab8c866615..28bcde5c140 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -7,6 +7,7 @@ #include #include +#include namespace DB { @@ -179,7 +180,7 @@ namespace detail /// Sorting an array will not be considered a violation of constancy. auto & array = elems; - std::nth_element(array.begin(), array.begin() + n, array.end()); + miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); quantile = array[n]; } @@ -200,7 +201,7 @@ namespace detail ? level * elems.size() : (elems.size() - 1); - std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); + miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); result[level_index] = array[n]; prev_n = n; diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index b27506c1cfb..4285259a4f4 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -163,10 +163,10 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum { const auto& [first, last] = equal_ranges[i]; if (reverse) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, + std::sort(res.begin() + first, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); else - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, + std::sort(res.begin() + first, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); auto new_first = first; From a4e000cc9f0877451864016ef1607a78c6a44de7 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 00:55:17 +0300 Subject: [PATCH 36/54] Fix name --- src/AggregateFunctions/QuantileExact.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index e3d1ae5d8b8..de909c27565 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -88,7 +88,7 @@ struct QuantileExact : QuantileExactBase> { size_t n = level < 1 ? level * array.size() : (array.size() - 1); - miniselect::floyd_rivest_partial_select(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. + miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. return array[n]; } From 5d138f3475a78a1eccf4b1b92c68344b5120e344 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 01:31:24 +0300 Subject: [PATCH 37/54] Fix submodules for fast test --- docker/test/fasttest/run.sh | 2 +- utils/check-style/check-include | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index eb737948aa6..36aa8baf252 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,7 +127,7 @@ function clone_submodules ( cd "$FASTTEST_SOURCE" -SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring) +SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring contrib/miniselect) git submodule sync git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" diff --git a/utils/check-style/check-include b/utils/check-style/check-include index 35f94d6e706..fc88f1b1b9f 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -19,6 +19,7 @@ inc="-I. \ -I./contrib/double-conversion \ -I./contrib/cityhash102/include \ -I./contrib/croaring \ +-I./contrib/miniselect \ -I./contrib/murmurhash/include \ -I./contrib/zookeeper/src/c/include \ -I./contrib/zookeeper/src/c/generated \ From 811c3e5cd18a19200edcf720b70754fc0536c07b Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 01:32:50 +0300 Subject: [PATCH 38/54] Fix submodules --- utils/check-style/check-include | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-include b/utils/check-style/check-include index fc88f1b1b9f..b4f105ed0cb 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -19,7 +19,7 @@ inc="-I. \ -I./contrib/double-conversion \ -I./contrib/cityhash102/include \ -I./contrib/croaring \ --I./contrib/miniselect \ +-I./contrib/miniselect/include \ -I./contrib/murmurhash/include \ -I./contrib/zookeeper/src/c/include \ -I./contrib/zookeeper/src/c/generated \ From ca51cf9235316ffee17af7cfec5b7e2a611bc3d5 Mon Sep 17 00:00:00 2001 From: fenglv Date: Tue, 10 Nov 2020 07:20:50 +0000 Subject: [PATCH 39/54] Make member function const to prevent -readability-make-member-function-const --- src/Interpreters/TreeRewriter.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8d3cb123955..51609cbaa23 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -63,7 +63,7 @@ struct CustomizeFunctionsData const String & customized_func_name; - void visit(ASTFunction & func, ASTPtr &) + void visit(ASTFunction & func, ASTPtr &) const { if (Poco::toLower(func.name) == func_name) { @@ -97,7 +97,7 @@ struct CustomizeFunctionsSuffixData const String & customized_func_suffix; - void visit(ASTFunction & func, ASTPtr &) + void visit(ASTFunction & func, ASTPtr &) const { if (endsWith(Poco::toLower(func.name), func_suffix)) { @@ -118,7 +118,7 @@ struct CustomizeAggregateFunctionsSuffixData const String & customized_func_suffix; - void visit(ASTFunction & func, ASTPtr &) + void visit(ASTFunction & func, ASTPtr &) const { const auto & instance = AggregateFunctionFactory::instance(); if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix)) From c0ae61e3061968267d7f8341e5af3823189acfdf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 10 Nov 2020 10:33:31 +0300 Subject: [PATCH 40/54] Fix stress test script --- docker/test/stress/stress | 9 --------- 1 file changed, 9 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 874dca751f3..458f78fcdb4 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -17,13 +17,6 @@ def get_skip_list_cmd(path): return '' -def run_perf_test(cmd, xmls_path, output_folder): - output_path = os.path.join(output_folder, "perf_stress_run.txt") - f = open(output_path, 'w') - p = Popen("{} --skip-tags=long --recursive --input-files {}".format(cmd, xmls_path), shell=True, stdout=f, stderr=f) - return p - - def get_options(i): options = "" if 0 < i: @@ -75,8 +68,6 @@ if __name__ == "__main__": args = parser.parse_args() func_pipes = [] - perf_process = None - perf_process = run_perf_test(args.perf_test_cmd, args.perf_test_xml_path, args.output_folder) func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit) logging.info("Will wait functests to finish") From 435f410aaa393667f6d107e0054db705d679c740 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 10:35:27 +0300 Subject: [PATCH 41/54] Restart C From 59c0a739c4a3ff4d157f64b5837f57c7aa448895 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 12:18:51 +0300 Subject: [PATCH 42/54] Add test to skiplist. --- src/Storages/IStorage.cpp | 1 - tests/queries/0_stateless/arcadia_skip_list.txt | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ddd5b6727dc..3b04c6cd632 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -80,7 +80,6 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons return result; } - Pipe IStorage::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 4c4f7b29d66..6420eadfc09 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -159,9 +159,10 @@ 01533_collate_in_nullable 01542_collate_in_array 01543_collate_in_tuple +01545_url_file_format_settings 01546_log_queries_min_query_duration_ms 01547_query_log_current_database 01548_query_log_query_execution_ms 01552_dict_fixedstring 01555_system_distribution_queue_mask -01557_max_parallel_replicas_no_sample.sql +01557_max_parallel_replicas_no_sample.sql \ No newline at end of file From 363c1e05c0776b723970e1d15c0f6bdb6002b89e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 12:35:05 +0300 Subject: [PATCH 43/54] Try fix tests. --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f8220002b38..d9b201a0390 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1344,6 +1344,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( pipes.emplace_back(std::move(source_processor)); } + if (pipes.empty()) + continue; + auto pipe = Pipe::unitePipes(std::move(pipes)); /// Drop temporary columns, added by 'sorting_key_expr' From 059357d51eaff84d0790abadc65d938ee45202e1 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 10 Nov 2020 13:23:46 +0300 Subject: [PATCH 44/54] `ALTER UPDATE/DELETE ... IN PARTITION` with partition pruning in `ReplicatedMergeTree` (#13403) Co-authored-by: Alexander Kazakov --- src/Interpreters/MutationsInterpreter.cpp | 100 +++++++++++--- src/Interpreters/MutationsInterpreter.h | 14 +- src/Parsers/ASTAlterQuery.cpp | 24 +++- src/Parsers/ASTAlterQuery.h | 2 +- src/Parsers/ExpressionListParsers.cpp | 14 ++ src/Parsers/ExpressionListParsers.h | 9 +- src/Parsers/ParserAlterQuery.cpp | 19 ++- src/Parsers/ParserAlterQuery.h | 10 +- src/Storages/MergeTree/BoolMask.h | 12 +- .../MergeTree/EphemeralLockInZooKeeper.cpp | 14 +- .../MergeTree/EphemeralLockInZooKeeper.h | 69 +++++++++- .../ReplicatedMergeTreeMutationEntry.h | 5 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 10 ++ src/Storages/MutationCommands.cpp | 7 +- src/Storages/MutationCommands.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 123 +++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 15 ++- .../__init__.py | 0 .../configs/cluster.xml | 16 +++ .../configs/logs_config.xml | 17 +++ .../test.py | 98 ++++++++++++++ 21 files changed, 493 insertions(+), 89 deletions(-) create mode 100644 tests/integration/test_mutations_in_partitions_of_merge_tree/__init__.py create mode 100644 tests/integration/test_mutations_in_partitions_of_merge_tree/configs/cluster.xml create mode 100644 tests/integration/test_mutations_in_partitions_of_merge_tree/configs/logs_config.xml create mode 100644 tests/integration/test_mutations_in_partitions_of_merge_tree/test.py diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3e7ebfec139..66cce64cff7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +33,7 @@ namespace DB namespace ErrorCodes { + extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int UNKNOWN_MUTATION_COMMAND; @@ -92,6 +94,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman if (finder_data.nondeterministic_function_name) return finder_data.nondeterministic_function_name; + /// Currently UPDATE and DELETE both always have predicates so we can use fallthrough [[fallthrough]]; } @@ -110,7 +113,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman return {}; } -ASTPtr prepareQueryAffectedAST(const std::vector & commands) +ASTPtr prepareQueryAffectedAST(const std::vector & commands, const StoragePtr & storage, const Context & context) { /// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query. /// The result can differ from the number of affected rows (e.g. if there is an UPDATE command that @@ -125,20 +128,23 @@ ASTPtr prepareQueryAffectedAST(const std::vector & commands) count_func->arguments = std::make_shared(); select->select()->children.push_back(count_func); - if (commands.size() == 1) - select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone()); - else + ASTs conditions; + for (const MutationCommand & command : commands) { - auto coalesced_predicates = std::make_shared(); - coalesced_predicates->name = "or"; - coalesced_predicates->arguments = std::make_shared(); - coalesced_predicates->children.push_back(coalesced_predicates->arguments); - - for (const MutationCommand & command : commands) - coalesced_predicates->arguments->children.push_back(command.predicate->clone()); + if (ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command, storage, context)) + conditions.push_back(std::move(condition)); + } + if (conditions.size() > 1) + { + auto coalesced_predicates = makeASTFunction("or"); + coalesced_predicates->arguments->children = std::move(conditions); select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates)); } + else if (conditions.size() == 1) + { + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(conditions.front())); + } return select; } @@ -167,8 +173,9 @@ ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_ } + bool isStorageTouchedByMutations( - StoragePtr storage, + const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, Context context_copy) @@ -176,16 +183,33 @@ bool isStorageTouchedByMutations( if (commands.empty()) return false; + bool all_commands_can_be_skipped = true; + auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); for (const MutationCommand & command : commands) { if (!command.predicate) /// The command touches all rows. return true; + + if (command.partition && !storage_from_merge_tree_data_part) + throw Exception("ALTER UPDATE/DELETE ... IN PARTITION is not supported for non-MergeTree tables", ErrorCodes::NOT_IMPLEMENTED); + + if (command.partition && storage_from_merge_tree_data_part) + { + const String partition_id = storage_from_merge_tree_data_part->getPartitionIDFromQuery(command.partition, context_copy); + if (partition_id == storage_from_merge_tree_data_part->getPartitionId()) + all_commands_can_be_skipped = false; + } + else + all_commands_can_be_skipped = false; } + if (all_commands_can_be_skipped) + return false; + context_copy.setSetting("max_streams_to_max_threads_ratio", 1); context_copy.setSetting("max_threads", 1); - ASTPtr select_query = prepareQueryAffectedAST(commands); + ASTPtr select_query = prepareQueryAffectedAST(commands, storage, context_copy); /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and and give it into ExpressionBlockInputStream @@ -202,9 +226,42 @@ bool isStorageTouchedByMutations( auto count = (*block.getByName("count()").column)[0].get(); return count != 0; - } + +ASTPtr getPartitionAndPredicateExpressionForMutationCommand( + const MutationCommand & command, + const StoragePtr & storage, + const Context & context +) +{ + ASTPtr partition_predicate_as_ast_func; + if (command.partition) + { + String partition_id; + + auto storage_merge_tree = std::dynamic_pointer_cast(storage); + auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); + if (storage_merge_tree) + partition_id = storage_merge_tree->getPartitionIDFromQuery(command.partition, context); + else if (storage_from_merge_tree_data_part) + partition_id = storage_from_merge_tree_data_part->getPartitionIDFromQuery(command.partition, context); + else + throw Exception("ALTER UPDATE/DELETE ... IN PARTITION is not supported for non-MergeTree tables", ErrorCodes::NOT_IMPLEMENTED); + + partition_predicate_as_ast_func = makeASTFunction("equals", + std::make_shared("_partition_id"), + std::make_shared(partition_id) + ); + } + + if (command.predicate && command.partition) + return makeASTFunction("and", command.predicate->clone(), std::move(partition_predicate_as_ast_func)); + else + return command.predicate ? command.predicate->clone() : partition_predicate_as_ast_func; +} + + MutationsInterpreter::MutationsInterpreter( StoragePtr storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -349,7 +406,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); - auto negated_predicate = makeASTFunction("isZeroOrNull", command.predicate->clone()); + auto negated_predicate = makeASTFunction("isZeroOrNull", getPartitionAndPredicateExpressionForMutationCommand(command)); stages.back().filters.push_back(negated_predicate); } else if (command.type == MutationCommand::UPDATE) @@ -387,7 +444,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) const auto & update_expr = kv.second; auto updated_column = makeASTFunction("CAST", makeASTFunction("if", - command.predicate->clone(), + getPartitionAndPredicateExpressionForMutationCommand(command), makeASTFunction("CAST", update_expr->clone(), type_literal), @@ -592,7 +649,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns); + auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot); if (context.hasQueryContext()) for (const auto & it : syntax_result->getScalars()) context.getQueryContext().addScalar(it.first, it.second); @@ -759,10 +816,10 @@ const Block & MutationsInterpreter::getUpdatedHeader() const size_t MutationsInterpreter::evaluateCommandsSize() { for (const MutationCommand & command : commands) - if (unlikely(!command.predicate)) /// The command touches all rows. + if (unlikely(!command.predicate && !command.partition)) /// The command touches all rows. return mutation_ast->size(); - return std::max(prepareQueryAffectedAST(commands)->size(), mutation_ast->size()); + return std::max(prepareQueryAffectedAST(commands, storage, context)->size(), mutation_ast->size()); } std::optional MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const @@ -783,6 +840,11 @@ std::optional MutationsInterpreter::getStorageSortDescriptionIf return sort_description; } +ASTPtr MutationsInterpreter::getPartitionAndPredicateExpressionForMutationCommand(const MutationCommand & command) const +{ + return DB::getPartitionAndPredicateExpressionForMutationCommand(command, storage, context); +} + bool MutationsInterpreter::Stage::isAffectingAllColumns(const Names & storage_columns) const { /// is subset diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 59d9e7657c3..18658e605ad 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -20,7 +20,17 @@ using QueryPipelinePtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( - StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, Context context_copy); + const StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & commands, + Context context_copy +); + +ASTPtr getPartitionAndPredicateExpressionForMutationCommand( + const MutationCommand & command, + const StoragePtr & storage, + const Context & context +); /// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs) /// to this data. @@ -59,6 +69,8 @@ private: std::optional getStorageSortDescriptionIfPossible(const Block & header) const; + ASTPtr getPartitionAndPredicateExpressionForMutationCommand(const MutationCommand & command) const; + StoragePtr storage; StorageMetadataPtr metadata_snapshot; MutationCommands commands; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 918bd4acaa7..d07e57eefae 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -90,7 +90,7 @@ void ASTAlterCommand::formatImpl( column->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } @@ -150,7 +150,7 @@ void ASTAlterCommand::formatImpl( index->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } @@ -161,7 +161,7 @@ void ASTAlterCommand::formatImpl( index->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } @@ -272,7 +272,15 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::DELETE) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE WHERE " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE" << (settings.hilite ? hilite_none : ""); + + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::UPDATE) @@ -280,6 +288,12 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UPDATE " << (settings.hilite ? hilite_none : ""); update_assignments->formatImpl(settings, state, frame); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } @@ -298,7 +312,7 @@ void ASTAlterCommand::formatImpl( << (settings.hilite ? hilite_none : ""); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 78e0c726ddf..65657e5ecfd 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -103,7 +103,7 @@ public: */ ASTPtr constraint; - /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. + /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries. * The value or ID of the partition is stored here. */ ASTPtr partition; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index ad03d949174..0f06a0d2480 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -55,6 +55,12 @@ const char * ParserComparisonExpression::operators[] = nullptr }; +const char * ParserComparisonExpression::overlapping_operators_to_skip[] = +{ + "IN PARTITION", + nullptr +}; + const char * ParserLogicalNotExpression::operators[] = { "NOT", "not", @@ -137,6 +143,14 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node /// try to find any of the valid operators const char ** it; + Expected stub; + for (it = overlapping_operators_to_skip; *it; ++it) + if (ParserKeyword{*it}.checkWithoutMoving(pos, stub)) + break; + + if (*it) + break; + for (it = operators; *it; it += 2) if (parseOperator(pos, *it, expected)) break; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index cf77b8b4da4..4e21eff7f0e 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -82,6 +82,7 @@ class ParserLeftAssociativeBinaryOperatorList : public IParserBase { private: Operators_t operators; + Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } }; ParserPtr first_elem_parser; ParserPtr remaining_elem_parser; @@ -93,6 +94,11 @@ public: { } + ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_) + : operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), first_elem_parser(std::move(first_elem_parser_)) + { + } + ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_, ParserPtr && remaining_elem_parser_) : operators(operators_), first_elem_parser(std::move(first_elem_parser_)), @@ -284,7 +290,8 @@ class ParserComparisonExpression : public IParserBase { private: static const char * operators[]; - ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique()}; + static const char * overlapping_operators_to_skip[]; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, overlapping_operators_to_skip, std::make_unique()}; protected: const char * getName() const override{ return "comparison expression"; } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index bb682455a99..7050614007e 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -79,7 +79,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_to_volume("TO VOLUME"); ParserKeyword s_to_table("TO TABLE"); - ParserKeyword s_delete_where("DELETE WHERE"); + ParserKeyword s_delete("DELETE"); ParserKeyword s_update("UPDATE"); ParserKeyword s_where("WHERE"); ParserKeyword s_to("TO"); @@ -506,8 +506,17 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MODIFY_SAMPLE_BY; } - else if (s_delete_where.ignore(pos, expected)) + else if (s_delete.ignore(pos, expected)) { + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + + if (!s_where.ignore(pos, expected)) + return false; + if (!parser_exp_elem.parse(pos, command->predicate, expected)) return false; @@ -518,6 +527,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_assignment_list.parse(pos, command->update_assignments, expected)) return false; + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + if (!s_where.ignore(pos, expected)) return false; diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index a0981c77ca6..514ef876430 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -10,7 +10,7 @@ namespace DB * ALTER TABLE [db.]name [ON CLUSTER cluster] * [ADD COLUMN [IF NOT EXISTS] col_name type [AFTER col_after],] * [DROP COLUMN [IF EXISTS] col_to_drop, ...] - * [CLEAR COLUMN [IF EXISTS] col_to_clear [IN PARTITION partition],] + * [CLEAR COLUMN [IF EXISTS] col_to_clear[ IN PARTITION partition],] * [MODIFY COLUMN [IF EXISTS] col_to_modify type, ...] * [RENAME COLUMN [IF EXISTS] col_name TO col_name] * [MODIFY PRIMARY KEY (a, b, c...)] @@ -19,8 +19,12 @@ namespace DB * [DROP|DETACH|ATTACH PARTITION|PART partition, ...] * [FETCH PARTITION partition FROM ...] * [FREEZE [PARTITION] [WITH NAME name]] - * [DELETE WHERE ...] - * [UPDATE col_name = expr, ... WHERE ...] + * [DELETE[ IN PARTITION partition] WHERE ...] + * [UPDATE col_name = expr, ...[ IN PARTITION partition] WHERE ...] + * [ADD INDEX [IF NOT EXISTS] index_name [AFTER index_name]] + * [DROP INDEX [IF EXISTS] index_name] + * [CLEAR INDEX [IF EXISTS] index_name IN PARTITION partition] + * [MATERIALIZE INDEX [IF EXISTS] index_name [IN PARTITION partition]] * ALTER LIVE VIEW [db.name] * [REFRESH] */ diff --git a/src/Storages/MergeTree/BoolMask.h b/src/Storages/MergeTree/BoolMask.h index 3538c581137..c26a0ed6c58 100644 --- a/src/Storages/MergeTree/BoolMask.h +++ b/src/Storages/MergeTree/BoolMask.h @@ -9,17 +9,17 @@ struct BoolMask BoolMask() {} BoolMask(bool can_be_true_, bool can_be_false_) : can_be_true(can_be_true_), can_be_false(can_be_false_) {} - BoolMask operator &(const BoolMask & m) + BoolMask operator &(const BoolMask & m) const { - return BoolMask(can_be_true && m.can_be_true, can_be_false || m.can_be_false); + return {can_be_true && m.can_be_true, can_be_false || m.can_be_false}; } - BoolMask operator |(const BoolMask & m) + BoolMask operator |(const BoolMask & m) const { - return BoolMask(can_be_true || m.can_be_true, can_be_false && m.can_be_false); + return {can_be_true || m.can_be_true, can_be_false && m.can_be_false}; } - BoolMask operator !() + BoolMask operator !() const { - return BoolMask(can_be_false, can_be_true); + return {can_be_false, can_be_true}; } /// If mask is (true, true), then it can no longer change under operation |. diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 6b00215fd26..1f194092f5f 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -71,13 +72,13 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( const String & block_numbers_path, const String & path_prefix, const String & temp_path, zkutil::ZooKeeper & zookeeper_) - : zookeeper(zookeeper_) + : zookeeper(&zookeeper_) { std::vector holders; while (true) { Coordination::Stat partitions_stat; - Strings partitions = zookeeper.getChildren(block_numbers_path, &partitions_stat); + Strings partitions = zookeeper->getChildren(block_numbers_path, &partitions_stat); if (holders.size() < partitions.size()) { @@ -85,7 +86,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( for (size_t i = 0; i < partitions.size() - holders.size(); ++i) { String path = temp_path + "/abandonable_lock-"; - holder_futures.push_back(zookeeper.asyncCreate(path, {}, zkutil::CreateMode::EphemeralSequential)); + holder_futures.push_back(zookeeper->asyncCreate(path, {}, zkutil::CreateMode::EphemeralSequential)); } for (auto & future : holder_futures) { @@ -104,7 +105,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( lock_ops.push_back(zkutil::makeCheckRequest(block_numbers_path, partitions_stat.version)); Coordination::Responses lock_responses; - Coordination::Error rc = zookeeper.tryMulti(lock_ops, lock_responses); + Coordination::Error rc = zookeeper->tryMulti(lock_ops, lock_responses); if (rc == Coordination::Error::ZBADVERSION) { LOG_TRACE(&Poco::Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); @@ -131,13 +132,16 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( void EphemeralLocksInAllPartitions::unlock() { + if (!zookeeper) + return; + std::vector futures; for (const auto & lock : locks) { Coordination::Requests unlock_ops; unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.path, -1)); unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.holder_path, -1)); - futures.push_back(zookeeper.asyncMulti(unlock_ops)); + futures.push_back(zookeeper->asyncMulti(unlock_ops)); } for (auto & future : futures) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index b85761e0b15..007768aea3a 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -1,9 +1,14 @@ #pragma once +#include "ReplicatedMergeTreeMutationEntry.h" + #include #include #include +#include +#include + namespace DB { @@ -87,13 +92,30 @@ private: /// Acquires block number locks in all partitions. -class EphemeralLocksInAllPartitions : private boost::noncopyable +class EphemeralLocksInAllPartitions : public boost::noncopyable { public: EphemeralLocksInAllPartitions( const String & block_numbers_path, const String & path_prefix, const String & temp_path, zkutil::ZooKeeper & zookeeper_); + EphemeralLocksInAllPartitions() = default; + + EphemeralLocksInAllPartitions(EphemeralLocksInAllPartitions && rhs) noexcept + : zookeeper(rhs.zookeeper) + , locks(std::move(rhs.locks)) + { + rhs.zookeeper = nullptr; + } + + EphemeralLocksInAllPartitions & operator=(EphemeralLocksInAllPartitions && rhs) noexcept + { + zookeeper = rhs.zookeeper; + rhs.zookeeper = nullptr; + locks = std::move(rhs.locks); + return *this; + } + struct LockInfo { String path; @@ -110,8 +132,51 @@ public: ~EphemeralLocksInAllPartitions(); private: - zkutil::ZooKeeper & zookeeper; + zkutil::ZooKeeper * zookeeper = nullptr; std::vector locks; }; + +/// This class allows scoped manipulations with block numbers locked in certain partitions +/// See StorageReplicatedMergeTree::allocateBlockNumbersInAffectedPartitions and alter()/mutate() methods +class PartitionBlockNumbersHolder +{ +public: + PartitionBlockNumbersHolder(const PartitionBlockNumbersHolder &) = delete; + PartitionBlockNumbersHolder & operator=(const PartitionBlockNumbersHolder &) = delete; + + using BlockNumbersType = ReplicatedMergeTreeMutationEntry::BlockNumbersType; + + PartitionBlockNumbersHolder() = default; + PartitionBlockNumbersHolder( + BlockNumbersType block_numbers_, std::optional locked_block_numbers_holder) + : block_numbers(std::move(block_numbers_)) + , multiple_partitions_holder(std::move(locked_block_numbers_holder)) + { + } + PartitionBlockNumbersHolder( + BlockNumbersType block_numbers_, std::optional locked_block_numbers_holder) + : block_numbers(std::move(block_numbers_)) + , single_partition_holder(std::move(locked_block_numbers_holder)) + { + } + + PartitionBlockNumbersHolder & operator=(PartitionBlockNumbersHolder &&) = default; + + const BlockNumbersType & getBlockNumbers() const { return block_numbers; } + + void reset() + { + multiple_partitions_holder.reset(); + single_partition_holder.reset(); + block_numbers.clear(); + } + +private: + BlockNumbersType block_numbers; + + std::optional multiple_partitions_holder; + std::optional single_partition_holder; +}; + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index cfcc3dec6f2..6229b13c47f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -35,9 +35,10 @@ struct ReplicatedMergeTreeMutationEntry /// Replica which initiated mutation String source_replica; - /// Accured numbers of blocks + /// Acquired block numbers /// partition_id -> block_number - std::map block_numbers; + using BlockNumbersType = std::map; + BlockNumbersType block_numbers; /// Mutation commands which will give to MUTATE_PART entries MutationCommands commands; diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index aa24ddcf33c..4c171053d61 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -45,6 +45,16 @@ public: return part->storage.getVirtuals(); } + String getPartitionId() const + { + return part->info.partition_id; + } + + String getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const + { + return part->storage.getPartitionIDFromQuery(ast, context); + } + protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) : IStorage(getIDFromPart(part_)) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 53c9b50cb9d..bb22a1d0395 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -2,11 +2,13 @@ #include #include #include -#include #include #include #include +#include +#include #include +#include #include #include #include @@ -32,6 +34,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.ast = command->ptr(); res.type = DELETE; res.predicate = command->predicate; + res.partition = command->partition; return res; } else if (command->type == ASTAlterCommand::UPDATE) @@ -40,6 +43,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.ast = command->ptr(); res.type = UPDATE; res.predicate = command->predicate; + res.partition = command->partition; for (const ASTPtr & assignment_ast : command->update_assignments->children) { const auto & assignment = assignment_ast->as(); @@ -124,6 +128,7 @@ std::shared_ptr MutationCommands::ast() const return res; } + void MutationCommands::writeText(WriteBuffer & out) const { std::stringstream commands_ss; diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 6e641e42cff..0f031eb56e6 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -43,8 +43,10 @@ struct MutationCommand /// Columns with corresponding actions std::unordered_map column_to_update_expression; - /// For MATERIALIZE INDEX + /// For MATERIALIZE INDEX. String index_name; + + /// For MATERIALIZE INDEX, UPDATE and DELETE. ASTPtr partition; /// For reads, drops and etc. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ac55df5885c..8c48febca1b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3915,6 +3915,60 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer } +std::set StorageReplicatedMergeTree::getPartitionIdsAffectedByCommands( + const MutationCommands & commands, const Context & query_context) const +{ + std::set affected_partition_ids; + + for (const auto & command : commands) + { + if (!command.partition) + { + affected_partition_ids.clear(); + break; + } + + affected_partition_ids.insert( + getPartitionIDFromQuery(command.partition, query_context) + ); + } + + return affected_partition_ids; +} + + +PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAffectedPartitions( + const MutationCommands & commands, const Context & query_context, const zkutil::ZooKeeperPtr & zookeeper) const +{ + const std::set mutation_affected_partition_ids = getPartitionIdsAffectedByCommands(commands, query_context); + + if (mutation_affected_partition_ids.size() == 1) + { + const auto & affected_partition_id = *mutation_affected_partition_ids.cbegin(); + auto block_number_holder = allocateBlockNumber(affected_partition_id, zookeeper); + if (!block_number_holder.has_value()) + return {}; + auto block_number = block_number_holder->getNumber(); /// Avoid possible UB due to std::move + return {{{affected_partition_id, block_number}}, std::move(block_number_holder)}; + } + else + { + /// TODO: Implement optimal block number aqcuisition algorithm in multiple (but not all) partitions + EphemeralLocksInAllPartitions lock_holder( + zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper); + + PartitionBlockNumbersHolder::BlockNumbersType block_numbers; + for (const auto & lock : lock_holder.getLocks()) + { + if (mutation_affected_partition_ids.empty() || mutation_affected_partition_ids.count(lock.partition_id)) + block_numbers[lock.partition_id] = lock.number; + } + + return {std::move(block_numbers), std::move(lock_holder)}; + } +} + + void StorageReplicatedMergeTree::alter( const AlterCommands & commands, const Context & query_context, TableLockHolder & table_lock_holder) { @@ -3942,7 +3996,7 @@ void StorageReplicatedMergeTree::alter( return queryToString(query); }; - auto zookeeper = getZooKeeper(); + const auto zookeeper = getZooKeeper(); std::optional alter_entry; std::optional mutation_znode; @@ -3953,10 +4007,6 @@ void StorageReplicatedMergeTree::alter( alter_entry.emplace(); mutation_znode.reset(); - /// We can safely read structure, because we guarded with alter_intention_lock - if (is_readonly) - throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); - auto current_metadata = getInMemoryMetadataPtr(); StorageInMemoryMetadata future_metadata = *current_metadata; @@ -4029,27 +4079,23 @@ void StorageReplicatedMergeTree::alter( ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); - std::optional lock_holder; - - /// Now we will prepare mutations record. - /// This code pretty same with mutate() function but process results slightly differently. + PartitionBlockNumbersHolder partition_block_numbers_holder; if (alter_entry->have_mutation) { - String mutations_path = zookeeper_path + "/mutations"; + const String mutations_path(zookeeper_path + "/mutations"); ReplicatedMergeTreeMutationEntry mutation_entry; - mutation_entry.source_replica = replica_name; - mutation_entry.commands = maybe_mutation_commands; mutation_entry.alter_version = new_metadata_version; + mutation_entry.source_replica = replica_name; + mutation_entry.commands = std::move(maybe_mutation_commands); + Coordination::Stat mutations_stat; zookeeper->get(mutations_path, &mutations_stat); - lock_holder.emplace( - zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper); - - for (const auto & lock : lock_holder->getLocks()) - mutation_entry.block_numbers[lock.partition_id] = lock.number; + partition_block_numbers_holder = + allocateBlockNumbersInAffectedPartitions(mutation_entry.commands, query_context, zookeeper); + mutation_entry.block_numbers = partition_block_numbers_holder.getBlockNumbers(); mutation_entry.create_time = time(nullptr); ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); @@ -4060,6 +4106,11 @@ void StorageReplicatedMergeTree::alter( Coordination::Responses results; Coordination::Error rc = zookeeper->tryMulti(ops, results); + /// For the sake of constitency with mechanics of concurrent background process of assigning parts merge tasks + /// this placeholder must be held up until the moment of committing into ZK of the mutation entry + /// See ReplicatedMergeTreeMergePredicate::canMergeTwoParts() method + partition_block_numbers_holder.reset(); + if (rc == Coordination::Error::ZOK) { if (alter_entry->have_mutation) @@ -4398,7 +4449,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c } -bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) +bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const { { std::lock_guard lock(existing_nodes_cache_mutex); @@ -4420,7 +4471,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) std::optional StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path) + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path) const { /// Lets check for duplicates in advance, to avoid superfluous block numbers allocation Coordination::Requests deduplication_check_ops; @@ -5063,44 +5114,46 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const /// After all needed parts are mutated (i.e. all active parts have the mutation version greater than /// the version of this mutation), the mutation is considered done and can be deleted. - ReplicatedMergeTreeMutationEntry entry; - entry.source_replica = replica_name; - entry.commands = commands; + ReplicatedMergeTreeMutationEntry mutation_entry; + mutation_entry.source_replica = replica_name; + mutation_entry.commands = commands; - String mutations_path = zookeeper_path + "/mutations"; + const String mutations_path = zookeeper_path + "/mutations"; + const auto zookeeper = getZooKeeper(); /// Update the mutations_path node when creating the mutation and check its version to ensure that /// nodes for mutations are created in the same order as the corresponding block numbers. /// Should work well if the number of concurrent mutation requests is small. while (true) { - auto zookeeper = getZooKeeper(); - Coordination::Stat mutations_stat; zookeeper->get(mutations_path, &mutations_stat); - EphemeralLocksInAllPartitions block_number_locks( - zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper); + PartitionBlockNumbersHolder partition_block_numbers_holder = + allocateBlockNumbersInAffectedPartitions(mutation_entry.commands, query_context, zookeeper); - for (const auto & lock : block_number_locks.getLocks()) - entry.block_numbers[lock.partition_id] = lock.number; - - entry.create_time = time(nullptr); + mutation_entry.block_numbers = partition_block_numbers_holder.getBlockNumbers(); + mutation_entry.create_time = time(nullptr); + /// The following version check guarantees the linearizability property for any pair of mutations: + /// mutation with higher sequence number is guaranteed to have higher block numbers in every partition + /// (and thus will be applied strictly according to sequence numbers of mutations) Coordination::Requests requests; requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); requests.emplace_back(zkutil::makeCreateRequest( - mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential)); + mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); Coordination::Responses responses; Coordination::Error rc = zookeeper->tryMulti(requests, responses); + partition_block_numbers_holder.reset(); + if (rc == Coordination::Error::ZOK) { const String & path_created = dynamic_cast(responses[1].get())->path_created; - entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1); - LOG_TRACE(log, "Created mutation with ID {}", entry.znode_name); + mutation_entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1); + LOG_TRACE(log, "Created mutation with ID {}", mutation_entry.znode_name); break; } else if (rc == Coordination::Error::ZBADVERSION) @@ -5112,7 +5165,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const throw Coordination::Exception("Unable to create a mutation znode", rc); } - waitMutation(entry.znode_name, query_context.getSettingsRef().mutations_sync); + waitMutation(mutation_entry.znode_name, query_context.getSettingsRef().mutations_sync); } void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) const diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5944c9ce3a8..c9f94427112 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -506,8 +506,8 @@ private: /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( - const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, - const String & zookeeper_block_id_path = ""); + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, + const String & zookeeper_block_id_path = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica . @@ -531,9 +531,9 @@ private: bool getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info, bool for_replace_partition = false); /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. - std::unordered_set existing_nodes_cache; - std::mutex existing_nodes_cache_mutex; - bool existsNodeCached(const std::string & path); + mutable std::unordered_set existing_nodes_cache; + mutable std::mutex existing_nodes_cache_mutex; + bool existsNodeCached(const std::string & path) const; void getClearBlocksInPartitionOps(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); /// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range. @@ -565,6 +565,11 @@ private: MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override; void startBackgroundMovesIfNeeded() override; + + std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, const Context & query_context) const; + PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( + const MutationCommands & commands, const Context & query_context, const zkutil::ZooKeeperPtr & zookeeper) const; + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ diff --git a/tests/integration/test_mutations_in_partitions_of_merge_tree/__init__.py b/tests/integration/test_mutations_in_partitions_of_merge_tree/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/cluster.xml b/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/cluster.xml new file mode 100644 index 00000000000..ec7c9b8e4f8 --- /dev/null +++ b/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/logs_config.xml b/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/logs_config.xml new file mode 100644 index 00000000000..bdf1bbc11c1 --- /dev/null +++ b/tests/integration/test_mutations_in_partitions_of_merge_tree/configs/logs_config.xml @@ -0,0 +1,17 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + + system + part_log
+ 500 +
+
diff --git a/tests/integration/test_mutations_in_partitions_of_merge_tree/test.py b/tests/integration/test_mutations_in_partitions_of_merge_tree/test.py new file mode 100644 index 00000000000..c7858c2f74d --- /dev/null +++ b/tests/integration/test_mutations_in_partitions_of_merge_tree/test.py @@ -0,0 +1,98 @@ +import pytest +import helpers.client +import helpers.cluster + + +cluster = helpers.cluster.ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', main_configs=['configs/logs_config.xml', 'configs/cluster.xml'], + with_zookeeper=True, stay_alive=True) + +node2 = cluster.add_instance('node2', main_configs=['configs/logs_config.xml', 'configs/cluster.xml'], + with_zookeeper=True, stay_alive=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_trivial_alter_in_partition_merge_tree_without_where(started_cluster): + try: + name = "test_trivial_alter_in_partition_merge_tree_without_where" + node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("CREATE TABLE {} (p Int64, x Int64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY p".format(name)) + node1.query("INSERT INTO {} VALUES (1, 2), (2, 3)".format(name)) + with pytest.raises(helpers.client.QueryRuntimeException): + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 1 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["5"] + with pytest.raises(helpers.client.QueryRuntimeException): + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["5"] + with pytest.raises(helpers.client.QueryRuntimeException): + node1.query("ALTER TABLE {} DELETE IN PARTITION 1 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["5"] + with pytest.raises(helpers.client.QueryRuntimeException): + node1.query("ALTER TABLE {} DELETE IN PARTITION 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["5"] + finally: + node1.query("DROP TABLE IF EXISTS {}".format(name)) + + +def test_trivial_alter_in_partition_merge_tree_with_where(started_cluster): + try: + name = "test_trivial_alter_in_partition_merge_tree_with_where" + node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("CREATE TABLE {} (p Int64, x Int64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY p".format(name)) + node1.query("INSERT INTO {} VALUES (1, 2), (2, 3)".format(name)) + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["6"] + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 1 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["6"] + node1.query("ALTER TABLE {} DELETE IN PARTITION 2 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["2"] + node1.query("ALTER TABLE {} DELETE IN PARTITION 1 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + assert node1.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["2"] + finally: + node1.query("DROP TABLE IF EXISTS {}".format(name)) + + +def test_trivial_alter_in_partition_replicated_merge_tree(started_cluster): + try: + name = "test_trivial_alter_in_partition_replicated_merge_tree" + + node1.query("DROP TABLE IF EXISTS {}".format(name)) + node2.query("DROP TABLE IF EXISTS {}".format(name)) + + for node in (node1, node2): + node.query( + "CREATE TABLE {name} (p Int64, x Int64) ENGINE=ReplicatedMergeTree('/clickhouse/{name}', '{{instance}}') ORDER BY tuple() PARTITION BY p" + .format(name=name)) + + node1.query("INSERT INTO {} VALUES (1, 2)".format(name)) + node2.query("INSERT INTO {} VALUES (2, 3)".format(name)) + + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 2 WHERE 1 SETTINGS mutations_sync = 2".format(name)) + for node in (node1, node2): + assert node.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["6"] + node1.query("ALTER TABLE {} UPDATE x = x + 1 IN PARTITION 1 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + for node in (node1, node2): + assert node.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["6"] + with pytest.raises(helpers.client.QueryRuntimeException): + node1.query("ALTER TABLE {} DELETE IN PARTITION 2 SETTINGS mutations_sync = 2".format(name)) + for node in (node1, node2): + assert node.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["6"] + node1.query("ALTER TABLE {} DELETE IN PARTITION 2 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + for node in (node1, node2): + assert node.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["2"] + node1.query("ALTER TABLE {} DELETE IN PARTITION 1 WHERE p = 2 SETTINGS mutations_sync = 2".format(name)) + for node in (node1, node2): + assert node.query("SELECT sum(x) FROM {}".format(name)).splitlines() == ["2"] + finally: + node1.query("DROP TABLE IF EXISTS {}".format(name)) + node2.query("DROP TABLE IF EXISTS {}".format(name)) From 046830967c80542535f64912fdab75ad9a6bc503 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 13:26:26 +0300 Subject: [PATCH 45/54] Try fix tests. --- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a257173d6b4..efd933c9a8b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -181,7 +181,8 @@ void StorageMergeTree::read( size_t max_block_size, unsigned num_streams) { - query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); + if (auto plan = reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)) + query_plan = std::move(*plan); } Pipe StorageMergeTree::read( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 394294507b1..faeb7e69022 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3629,11 +3629,13 @@ void StorageReplicatedMergeTree::read( if (context.getSettingsRef().select_sequential_consistency) { auto max_added_blocks = getMaxAddedBlocks(); - query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams, &max_added_blocks)); + if (auto plan = reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams, &max_added_blocks)) + query_plan = std::move(*plan); return; } - query_plan = std::move(*reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); + if (auto plan = reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)) + query_plan = std::move(*plan); } Pipe StorageReplicatedMergeTree::read( From 07fe3a6347f7d5a83d4a374adac9a531062c5b59 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 15:14:05 +0300 Subject: [PATCH 46/54] Fix build. --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index dd8f51aaa72..7f031771373 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -79,17 +79,6 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts return Block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "_part")}; } -/// Check if ORDER BY clause of the query has some expression. -static bool sortingDescriptionHasExpressions(const SortDescription & sort_description, const StorageMetadataPtr & metadata_snapshot) -{ - auto all_columns = metadata_snapshot->getColumns(); - for (const auto & sort_column : sort_description) - { - if (!all_columns.has(sort_column.column_name)) - return true; - } - return false; -} size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, From b659efdb437abb91ae4a7b7544013d6439ccd46e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 16:18:36 +0300 Subject: [PATCH 47/54] Fix test. --- ...1_mergetree_read_in_order_spread.reference | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference index fc10b4707a9..adbb03ae018 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -8,4 +8,23 @@ ExpressionTransform AggregatingInOrderTransform × 3 (Expression) ExpressionTransform × 3 - (ReadFromStorage) + (SettingQuotaAndLimits) + (Expression) + ExpressionTransform × 3 + (Union) + (MergingSorted) + (Expression) + ExpressionTransform + (ReadFromStorage) + MergeTree 0 → 1 + (MergingSorted) + MergingSortedTransform 2 → 1 + (Expression) + ExpressionTransform × 2 + (ReadFromStorage) + MergeTree × 2 0 → 1 + (MergingSorted) + (Expression) + ExpressionTransform + (ReadFromStorage) + MergeTree 0 → 1 From 115807e77add0c46355530a2672991af44bcba1a Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 10 Nov 2020 17:54:53 +0300 Subject: [PATCH 48/54] DOCSUP-3043: Document the null function (ru) (#16795) * my changes to gitignore * Corrections in English docs and Russian docs added. * TOC corrections * TOC fixed * Revert "my changes to gitignore" This reverts commit 5884b1e79b0eb40d9c39b019d345d9dbc3c45640. * Update docs/en/sql-reference/table-functions/null.md * Update docs/en/sql-reference/table-functions/null.md * Update docs/ru/sql-reference/table-functions/null.md * Update docs/ru/sql-reference/table-functions/null.md Co-authored-by: Olga Revyakina Co-authored-by: BayoNet --- docs/en/sql-reference/table-functions/null.md | 8 ++-- docs/ru/sql-reference/table-functions/null.md | 43 +++++++++++++++++++ 2 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 docs/ru/sql-reference/table-functions/null.md diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md index 6edec61add7..355a45a83e1 100644 --- a/docs/en/sql-reference/table-functions/null.md +++ b/docs/en/sql-reference/table-functions/null.md @@ -5,7 +5,7 @@ toc_title: null function # null {#null-function} -Accepts an inserted data of the specified structure and immediately drops it away. The function is used for convenience writing tests and demonstrations. +Creates a temporary table of the specified structure with the [Null](../../engines/table-engines/special/null.md) table engine. According to the `Null`-engine properties, the table data is ignored and the table itself is immediately droped right after the query execution. The function is used for the convenience of test writing and demonstrations. **Syntax** @@ -19,7 +19,7 @@ null('structure') **Returned value** -A table with the specified structure, which is dropped right after the query execution. +A temporary `Null`-engine table with the specified structure. **Example** @@ -36,6 +36,8 @@ INSERT INTO t SELECT * FROM numbers_mt(1000000000); DROP TABLE IF EXISTS t; ``` -See also: format **Null**. +See also: + +- [Null table engine](../../engines/table-engines/special/null.md) [Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/null/) diff --git a/docs/ru/sql-reference/table-functions/null.md b/docs/ru/sql-reference/table-functions/null.md new file mode 100644 index 00000000000..8e0173733f8 --- /dev/null +++ b/docs/ru/sql-reference/table-functions/null.md @@ -0,0 +1,43 @@ +--- +toc_priority: 53 +toc_title: null функция +--- + +# null {#null-function} + +Создает временную таблицу указанной структуры с движком [Null](../../engines/table-engines/special/null.md). В соответствии со свойствами движка, данные в таблице игнорируются, а сама таблица удаляется сразу после выполнения запроса. Функция используется для удобства написания тестов и демонстрационных примеров. + +**Синтаксис** + +``` sql +null('structure') +``` + +**Параметр** + +- `structure` — список колонок и их типов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +Временная таблица указанной структуры с движком `Null`. + +**Пример** + +Один запрос с функцией `null`: + +``` sql +INSERT INTO function null('x UInt64') SELECT * FROM numbers_mt(1000000000); +``` +заменяет три запроса: + +```sql +CREATE TABLE t (x UInt64) ENGINE = Null; +INSERT INTO t SELECT * FROM numbers_mt(1000000000); +DROP TABLE IF EXISTS t; +``` + +См. также: + +- [Движок таблиц Null](../../engines/table-engines/special/null.md) + +[Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/null/) From 205719877945167b1615fcfa15f502310e55867d Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 17:58:05 +0300 Subject: [PATCH 49/54] Fix performance tests that became very fast --- tests/performance/push_down_limit.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 0dcd9335a52..6fc60bc5768 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,4 +1,4 @@ - select number from (select number from numbers(10000000) order by -number) limit 10 - select number from (select number from numbers_mt(100000000) order by -number) limit 10 + select number from (select number from numbers(150000000) order by -number) limit 10 + select number from (select number from numbers_mt(1500000000) order by -number) limit 10 From 8504efde772ffb7b08f56b6c52f77c2b257778ee Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 18:12:49 +0300 Subject: [PATCH 50/54] Merge --- tests/performance/push_down_limit.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 6fc60bc5768..4aa30cb9d5d 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,4 +1,4 @@ - + select number from (select number from numbers(150000000) order by -number) limit 10 select number from (select number from numbers_mt(1500000000) order by -number) limit 10 From 847fa9ca3cfa2e18233eb83da9975f41d86d8153 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 18:15:26 +0300 Subject: [PATCH 51/54] Merge --- tests/performance/push_down_limit.xml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 4aa30cb9d5d..549882bf7a7 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,4 +1,8 @@ - - select number from (select number from numbers(150000000) order by -number) limit 10 - select number from (select number from numbers_mt(1500000000) order by -number) limit 10 + + CREATE VIEW numbers_view AS SELECT number from numbers_mt(100000000) order by number desc + + select number from (select number from numbers(1500000000) order by -number) limit 10 + select number from (select number from numbers_mt(15000000000) order by -number) limit 10 + + select number from numbers_view limit 100 From 0ef6bd61880c2924e282dbe1992738b0936e0818 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 18:19:46 +0300 Subject: [PATCH 52/54] Fix tests, I cannot count number of zeros sometimes --- tests/performance/push_down_limit.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 549882bf7a7..0af6eec8b32 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,8 +1,8 @@ CREATE VIEW numbers_view AS SELECT number from numbers_mt(100000000) order by number desc - select number from (select number from numbers(1500000000) order by -number) limit 10 - select number from (select number from numbers_mt(15000000000) order by -number) limit 10 + select number from (select number from numbers(150000000) order by -number) limit 10 + select number from (select number from numbers_mt(1500000000) order by -number) limit 10 select number from numbers_view limit 100 From c0308a5d85f66932822553b4daabb9c999511689 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 18:21:39 +0300 Subject: [PATCH 53/54] Fix tests, I cannot count number of zeros sometimes --- tests/performance/push_down_limit.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 549882bf7a7..0af6eec8b32 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,8 +1,8 @@ CREATE VIEW numbers_view AS SELECT number from numbers_mt(100000000) order by number desc - select number from (select number from numbers(1500000000) order by -number) limit 10 - select number from (select number from numbers_mt(15000000000) order by -number) limit 10 + select number from (select number from numbers(150000000) order by -number) limit 10 + select number from (select number from numbers_mt(1500000000) order by -number) limit 10 select number from numbers_view limit 100 From c36e6fe37849d3b1f506476eab47e04324ac074e Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 18:22:59 +0300 Subject: [PATCH 54/54] Fix tests finally --- tests/performance/push_down_limit.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 0af6eec8b32..6ae63b54ec6 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -1,7 +1,7 @@ CREATE VIEW numbers_view AS SELECT number from numbers_mt(100000000) order by number desc - select number from (select number from numbers(150000000) order by -number) limit 10 + select number from (select number from numbers(1500000000) order by -number) limit 10 select number from (select number from numbers_mt(1500000000) order by -number) limit 10 select number from numbers_view limit 100